diff --git a/client/servicediscovery/eru_service_discovery.go b/client/servicediscovery/eru_service_discovery.go index 8204dd373..094473716 100644 --- a/client/servicediscovery/eru_service_discovery.go +++ b/client/servicediscovery/eru_service_discovery.go @@ -51,7 +51,7 @@ func (w *EruServiceDiscovery) Watch(ctx context.Context) (_ <-chan []string, err for { cancelTimer := make(chan struct{}) - go func() { + go func(expectedInterval time.Duration) { timer := time.NewTimer(expectedInterval * time.Second) defer timer.Stop() select { @@ -60,7 +60,7 @@ func (w *EruServiceDiscovery) Watch(ctx context.Context) (_ <-chan []string, err case <-cancelTimer: return } - }() + }(expectedInterval) status, err := stream.Recv() close(cancelTimer) if err != nil { diff --git a/cluster/calcium/create.go b/cluster/calcium/create.go index 8ae4017e7..3b799e789 100644 --- a/cluster/calcium/create.go +++ b/cluster/calcium/create.go @@ -42,7 +42,7 @@ func (c *Calcium) doCreateWorkloads(ctx context.Context, opts *types.DeployOptio var ( err error - planMap map[types.ResourceType]resourcetypes.ResourcePlans + plans []resourcetypes.ResourcePlans deployMap map[string]int rollbackMap map[string][]int ) @@ -71,14 +71,14 @@ func (c *Calcium) doCreateWorkloads(ctx context.Context, opts *types.DeployOptio }() // calculate plans - if planMap, deployMap, err = c.doAllocResource(ctx, nodeMap, opts); err != nil { + if plans, deployMap, err = c.doAllocResource(ctx, nodeMap, opts); err != nil { return errors.WithStack(err) } // commit changes nodes := []*types.Node{} for nodename, deploy := range deployMap { - for _, plan := range planMap { + for _, plan := range plans { plan.ApplyChangesOnNode(nodeMap[nodename], utils.Range(deploy)...) } nodes = append(nodes, nodeMap[nodename]) @@ -92,17 +92,16 @@ func (c *Calcium) doCreateWorkloads(ctx context.Context, opts *types.DeployOptio // then: deploy containers func(ctx context.Context) error { - rollbackMap, err = c.doDeployWorkloads(ctx, ch, opts, planMap, deployMap) + rollbackMap, err = c.doDeployWorkloads(ctx, ch, opts, plans, deployMap) return errors.WithStack(err) }, // rollback: give back resources - func(ctx context.Context) (err error) { - for nodeName, rollbackIndices := range rollbackMap { - indices := rollbackIndices - if e := c.withNodeLocked(ctx, nodeName, func(node *types.Node) error { - for _, plan := range planMap { - plan.RollbackChangesOnNode(node, indices...) + func(ctx context.Context, _ bool) (err error) { + for nodename, rollbackIndices := range rollbackMap { + if e := c.withNodeLocked(ctx, nodename, func(node *types.Node) error { + for _, plan := range plans { + plan.RollbackChangesOnNode(node, rollbackIndices...) // nolint:scopelint } return errors.WithStack(c.store.UpdateNodes(ctx, node)) }); e != nil { @@ -121,7 +120,7 @@ func (c *Calcium) doCreateWorkloads(ctx context.Context, opts *types.DeployOptio return ch, err } -func (c *Calcium) doDeployWorkloads(ctx context.Context, ch chan *types.CreateContainerMessage, opts *types.DeployOptions, planMap map[types.ResourceType]resourcetypes.ResourcePlans, deployMap map[string]int) (_ map[string][]int, err error) { +func (c *Calcium) doDeployWorkloads(ctx context.Context, ch chan *types.CreateContainerMessage, opts *types.DeployOptions, plans []resourcetypes.ResourcePlans, deployMap map[string]int) (_ map[string][]int, err error) { wg := sync.WaitGroup{} wg.Add(len(deployMap)) @@ -131,7 +130,7 @@ func (c *Calcium) doDeployWorkloads(ctx context.Context, ch chan *types.CreateCo go metrics.Client.SendDeployCount(deploy) go func(nodename string, deploy, seq int) { defer wg.Done() - if indices, e := c.doDeployWorkloadsOnNode(ctx, ch, nodename, opts, deploy, planMap, seq); e != nil { + if indices, e := c.doDeployWorkloadsOnNode(ctx, ch, nodename, opts, deploy, plans, seq); e != nil { err = e rollbackMap[nodename] = indices } @@ -145,8 +144,8 @@ func (c *Calcium) doDeployWorkloads(ctx context.Context, ch chan *types.CreateCo } // deploy scheduled containers on one node -func (c *Calcium) doDeployWorkloadsOnNode(ctx context.Context, ch chan *types.CreateContainerMessage, nodeName string, opts *types.DeployOptions, deploy int, planMap map[types.ResourceType]resourcetypes.ResourcePlans, seq int) (indices []int, err error) { - node, err := c.doGetAndPrepareNode(ctx, nodeName, opts.Image) +func (c *Calcium) doDeployWorkloadsOnNode(ctx context.Context, ch chan *types.CreateContainerMessage, nodename string, opts *types.DeployOptions, deploy int, plans []resourcetypes.ResourcePlans, seq int) (indices []int, err error) { + node, err := c.doGetAndPrepareNode(ctx, nodename, opts.Image) if err != nil { for i := 0; i < deploy; i++ { ch <- &types.CreateContainerMessage{Error: err} @@ -157,12 +156,11 @@ func (c *Calcium) doDeployWorkloadsOnNode(ctx context.Context, ch chan *types.Cr for idx := 0; idx < deploy; idx++ { createMsg := &types.CreateContainerMessage{ Podname: opts.Podname, - Nodename: nodeName, + Nodename: nodename, Publish: map[string][]string{}, } - func() { - var e error + do := func(idx int) (e error) { defer func() { if e != nil { err = e @@ -172,20 +170,21 @@ func (c *Calcium) doDeployWorkloadsOnNode(ctx context.Context, ch chan *types.Cr ch <- createMsg }() - rsc := &types.Resources{} + r := &types.ResourceMeta{} o := resourcetypes.DispenseOptions{ Node: node, Index: idx, } - for _, plan := range planMap { - if e = plan.Dispense(o, rsc); e != nil { + for _, plan := range plans { + if r, e = plan.Dispense(o, r); e != nil { return } } - createMsg.Resources = *rsc - e = c.doDeployOneWorkload(ctx, node, opts, createMsg, seq+idx, deploy-1-idx) - }() + createMsg.ResourceMeta = *r + return c.doDeployOneWorkload(ctx, node, opts, createMsg, seq+idx, deploy-1-idx) + } + _ = do(idx) } return indices, errors.WithStack(err) @@ -210,30 +209,29 @@ func (c *Calcium) doDeployOneWorkload( ) (err error) { config := c.doMakeContainerOptions(no, msg, opts, node) container := &types.Container{ - Name: config.Name, - Labels: config.Labels, - Podname: opts.Podname, - Nodename: node.Name, - CPURequest: msg.CPURequest, - CPULimit: msg.CPULimit, - QuotaRequest: msg.CPUQuotaRequest, - QuotaLimit: msg.CPUQuotaLimit, - MemoryRequest: msg.MemoryRequest, - MemoryLimit: msg.MemoryLimit, - StorageRequest: msg.StorageRequest, - StorageLimit: msg.StorageLimit, - VolumeRequest: msg.VolumeRequest, - VolumeLimit: msg.VolumeLimit, - VolumePlanRequest: msg.VolumePlanRequest, - VolumePlanLimit: msg.VolumePlanLimit, - Hook: opts.Entrypoint.Hook, - Privileged: opts.Entrypoint.Privileged, - Engine: node.Engine, - SoftLimit: opts.SoftLimit, - Image: opts.Image, - Env: opts.Env, - User: opts.User, - ResourceSubdivisible: true, + ResourceMeta: types.ResourceMeta{ + CPU: msg.CPU, + CPUQuotaRequest: msg.CPUQuotaRequest, + CPUQuotaLimit: msg.CPUQuotaLimit, + MemoryRequest: msg.MemoryRequest, + MemoryLimit: msg.MemoryLimit, + StorageRequest: msg.StorageRequest, + StorageLimit: msg.StorageLimit, + VolumeRequest: msg.VolumeRequest, + VolumeLimit: msg.VolumeLimit, + VolumePlanRequest: msg.VolumePlanRequest, + VolumePlanLimit: msg.VolumePlanLimit, + }, + Name: config.Name, + Labels: config.Labels, + Podname: opts.Podname, + Nodename: node.Name, + Hook: opts.Entrypoint.Hook, + Privileged: opts.Entrypoint.Privileged, + Engine: node.Engine, + Image: opts.Image, + Env: opts.Env, + User: opts.User, } return utils.Txn( ctx, @@ -308,7 +306,7 @@ func (c *Calcium) doDeployOneWorkload( }, // remove container - func(ctx context.Context) error { + func(ctx context.Context, _ bool) error { return errors.WithStack(c.doRemoveContainer(ctx, container, true)) }, c.config.GlobalTimeout, @@ -318,12 +316,11 @@ func (c *Calcium) doDeployOneWorkload( func (c *Calcium) doMakeContainerOptions(no int, msg *types.CreateContainerMessage, opts *types.DeployOptions, node *types.Node) *enginetypes.VirtualizationCreateOptions { config := &enginetypes.VirtualizationCreateOptions{} // general - config.CPU = msg.CPULimit + config.CPU = msg.CPU config.Quota = msg.CPUQuotaLimit config.Memory = msg.MemoryLimit config.Storage = msg.StorageLimit - config.NUMANode = node.GetNUMANode(msg.CPULimit) - config.SoftLimit = opts.SoftLimit + config.NUMANode = msg.NUMANode config.RawArgs = opts.RawArgs config.Lambda = opts.Lambda config.User = opts.User diff --git a/cluster/calcium/create_test.go b/cluster/calcium/create_test.go index 5fd155e8b..7aa3efedd 100644 --- a/cluster/calcium/create_test.go +++ b/cluster/calcium/create_test.go @@ -39,7 +39,7 @@ func TestCreateContainer(t *testing.T) { opts.Count = 1 // failed by memory check - opts.RawResourceOptions = types.RawResourceOptions{MemoryLimit: -1} + opts.ResourceOpts = types.ResourceOptions{MemoryLimit: -1} store.On("GetNodesByPod", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, nil) ch, err := c.CreateContainer(ctx, opts) assert.Nil(t, err) @@ -48,7 +48,7 @@ func TestCreateContainer(t *testing.T) { } // failed by CPUQuota - opts.RawResourceOptions = types.RawResourceOptions{CPULimit: -1, MemoryLimit: 1} + opts.ResourceOpts = types.ResourceOptions{CPUQuotaLimit: -1, MemoryLimit: 1} ch, err = c.CreateContainer(ctx, opts) assert.Nil(t, err) for m := range ch { @@ -60,12 +60,12 @@ func TestCreateContainerTxn(t *testing.T) { c := NewTestCluster() ctx := context.Background() opts := &types.DeployOptions{ - Count: 2, - DeployStrategy: strategy.Auto, - Podname: "p1", - RawResourceOptions: types.RawResourceOptions{CPULimit: 1}, - Image: "zc:test", - Entrypoint: &types.Entrypoint{}, + Count: 2, + DeployStrategy: strategy.Auto, + Podname: "p1", + ResourceOpts: types.ResourceOptions{CPUQuotaLimit: 1}, + Image: "zc:test", + Entrypoint: &types.Entrypoint{}, } store := &storemocks.Store{} sche := &schedulermocks.Scheduler{} diff --git a/cluster/calcium/dissociate.go b/cluster/calcium/dissociate.go index e7a453db2..0a3fa3ad2 100644 --- a/cluster/calcium/dissociate.go +++ b/cluster/calcium/dissociate.go @@ -26,7 +26,7 @@ func (c *Calcium) DissociateContainer(ctx context.Context, IDs []string) (chan * // then func(ctx context.Context) error { log.Infof("[DissociateContainer] Container %s dissociated", container.ID) - return c.store.UpdateNodeResource(ctx, node, container.CPURequest, container.QuotaRequest, container.MemoryRequest, container.StorageRequest, container.VolumePlanRequest.IntoVolumeMap(), store.ActionIncr) + return c.store.UpdateNodeResource(ctx, node, &container.ResourceMeta, store.ActionIncr) }, // rollback nil, diff --git a/cluster/calcium/dissociate_test.go b/cluster/calcium/dissociate_test.go index 82d459875..a3eae85d8 100644 --- a/cluster/calcium/dissociate_test.go +++ b/cluster/calcium/dissociate_test.go @@ -23,14 +23,16 @@ func TestDissociateContainer(t *testing.T) { lock.On("Unlock", mock.Anything).Return(nil) c1 := &types.Container{ - ID: "c1", - Podname: "p1", - MemoryLimit: 5 * int64(units.MiB), - MemoryRequest: 5 * int64(units.MiB), - QuotaLimit: 0.9, - QuotaRequest: 0.9, - CPURequest: types.CPUMap{"2": 90}, - Nodename: "node1", + ResourceMeta: types.ResourceMeta{ + MemoryLimit: 5 * int64(units.MiB), + MemoryRequest: 5 * int64(units.MiB), + CPUQuotaLimit: 0.9, + CPUQuotaRequest: 0.9, + CPU: types.CPUMap{"2": 90}, + }, + ID: "c1", + Podname: "p1", + Nodename: "node1", } node1 := &types.Node{ @@ -59,7 +61,7 @@ func TestDissociateContainer(t *testing.T) { } store.On("RemoveContainer", mock.Anything, mock.Anything).Return(nil) // success - store.On("UpdateNodeResource", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) + store.On("UpdateNodeResource", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) ch, err = c.DissociateContainer(ctx, []string{"c1"}) assert.NoError(t, err) for r := range ch { diff --git a/cluster/calcium/realloc.go b/cluster/calcium/realloc.go index 275980e55..743378753 100644 --- a/cluster/calcium/realloc.go +++ b/cluster/calcium/realloc.go @@ -2,7 +2,6 @@ package calcium import ( "context" - "sync" "github.com/pkg/errors" enginetypes "github.com/projecteru2/core/engine/types" @@ -10,282 +9,129 @@ import ( resourcetypes "github.com/projecteru2/core/resources/types" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" - log "github.com/sirupsen/logrus" ) -// nodename -> container list -type nodeContainers map[string][]*types.Container - -// ReallocResource allow realloc container resource -func (c *Calcium) ReallocResource(ctx context.Context, opts *types.ReallocOptions) (chan *types.ReallocResourceMessage, error) { - ch := make(chan *types.ReallocResourceMessage) - go func() { - defer close(ch) - if err := c.withContainersLocked(ctx, opts.IDs, func(containers map[string]*types.Container) error { - // Pod-Node-Containers - containersInfo := map[*types.Pod]nodeContainers{} - // Pod cache - podCache := map[string]*types.Pod{} - var err error - for _, container := range containers { - pod, ok := podCache[container.Podname] - if !ok { - pod, err = c.store.GetPod(ctx, container.Podname) - if err != nil { - ch <- &types.ReallocResourceMessage{ - ContainerID: container.ID, - Error: err, - } - continue - } - podCache[container.Podname] = pod - containersInfo[pod] = nodeContainers{} - } - if _, ok = containersInfo[pod][container.Nodename]; !ok { - containersInfo[pod][container.Nodename] = []*types.Container{} - } - containersInfo[pod][container.Nodename] = append(containersInfo[pod][container.Nodename], container) - } - - wg := sync.WaitGroup{} - wg.Add(len(containersInfo)) - // deal with normal container - for _, nodeContainersInfo := range containersInfo { - go func(nodeContainersInfo nodeContainers) { - defer wg.Done() - c.doReallocContainersOnPod(ctx, ch, nodeContainersInfo, opts) - }(nodeContainersInfo) - } - wg.Wait() - return nil - }); err != nil { - log.Errorf("[ReallocResource] Realloc failed %+v", err) - for _, ID := range opts.IDs { - ch <- &types.ReallocResourceMessage{ - ContainerID: ID, - Error: err, - } - } +// ReallocResource updates workload resource dynamically +func (c *Calcium) ReallocResource(ctx context.Context, opts *types.ReallocOptions) (err error) { + return c.withContainerLocked(ctx, opts.ID, func(container *types.Container) error { + rrs, err := resources.MakeRequests( + types.ResourceOptions{ + CPUQuotaRequest: container.CPUQuotaRequest + opts.ResourceOpts.CPUQuotaRequest, + CPUQuotaLimit: container.CPUQuotaLimit + opts.ResourceOpts.CPUQuotaLimit, + CPUBind: types.ParseTriOption(opts.CPUBindOpts, len(container.CPU) > 0), + MemoryRequest: container.MemoryRequest + opts.ResourceOpts.MemoryRequest, + MemoryLimit: container.MemoryLimit + opts.ResourceOpts.MemoryLimit, + StorageRequest: container.StorageRequest + opts.ResourceOpts.StorageRequest, + StorageLimit: container.StorageLimit + opts.ResourceOpts.StorageLimit, + VolumeRequest: types.MergeVolumeBindings(container.VolumeRequest, opts.ResourceOpts.VolumeRequest), + VolumeLimit: types.MergeVolumeBindings(container.VolumeLimit, opts.ResourceOpts.VolumeLimit), + }, + ) + if err != nil { + return errors.WithStack(err) } - }() - return ch, nil + return c.doReallocOnNode(ctx, container.Nodename, container, rrs) + }) } -// group containers by node and requests -func (c *Calcium) doReallocContainersOnPod(ctx context.Context, ch chan *types.ReallocResourceMessage, nodeContainersInfo nodeContainers, opts *types.ReallocOptions) { - hardVbsMap := map[string]types.VolumeBindings{} - containerGroups := map[string]map[resourcetypes.ResourceRequirements][]*types.Container{} - for nodename, containers := range nodeContainersInfo { - containerGroups[nodename] = map[resourcetypes.ResourceRequirements][]*types.Container{} - for _, container := range containers { - if err := func() (err error) { - var ( - autoVbsRequest, autoVbsLimit types.VolumeBindings - rrs resourcetypes.ResourceRequirements - ) - autoVbsRequest, hardVbsMap[container.ID] = types.MergeVolumeBindings(container.VolumeRequest, opts.VolumeRequest, opts.Volumes).Divide() - autoVbsLimit, _ = types.MergeVolumeBindings(container.VolumeLimit, opts.VolumeLimit, opts.Volumes).Divide() - - rrs, err = resources.NewResourceRequirements(types.RawResourceOptions{ - CPURequest: container.QuotaRequest + opts.CPURequest + opts.CPU, - CPULimit: container.QuotaLimit + opts.CPULimit + opts.CPU, - CPUBind: types.ParseTriOption(opts.BindCPUOpt, len(container.CPURequest) > 0), - MemoryRequest: container.MemoryRequest + opts.MemoryRequest + opts.Memory, - MemoryLimit: container.MemoryLimit + opts.MemoryLimit + opts.Memory, - MemorySoft: types.ParseTriOption(opts.MemoryLimitOpt, container.SoftLimit), - VolumeRequest: autoVbsRequest, - VolumeLimit: autoVbsLimit, - StorageRequest: container.StorageRequest + opts.StorageRequest + opts.Storage, - StorageLimit: container.StorageLimit + opts.StorageLimit + opts.Storage, - }) - - containerGroups[nodename][rrs] = append(containerGroups[nodename][rrs], container) - return - - }(); err != nil { - log.Errorf("[ReallocResource.doReallocContainersOnPod] Realloc failed: %+v", err) - ch <- &types.ReallocResourceMessage{Error: err} - return - } +// transaction: node resource +func (c *Calcium) doReallocOnNode(ctx context.Context, nodename string, container *types.Container, rrs resourcetypes.ResourceRequests) error { + return c.withNodeLocked(ctx, nodename, func(node *types.Node) error { + node.RecycleResources(&container.ResourceMeta) + _, total, plans, err := resources.SelectNodesByResourceRequests(rrs, map[string]*types.Node{node.Name: node}) + if err != nil { + return errors.WithStack(err) } - } - - for nodename, containerByApps := range containerGroups { - for rrs, containers := range containerByApps { - if err := c.doReallocContainersOnNode(ctx, ch, nodename, containers, rrs, hardVbsMap); err != nil { - - log.Errorf("[ReallocResource.doReallocContainersOnPod] Realloc failed: %+v", err) - ch <- &types.ReallocResourceMessage{Error: err} - } + if total < 1 { + return errors.WithStack(types.ErrInsufficientRes) } - } -} - -// transaction: node meta -func (c *Calcium) doReallocContainersOnNode(ctx context.Context, ch chan *types.ReallocResourceMessage, nodename string, containers []*types.Container, rrs resourcetypes.ResourceRequirements, hardVbsMap map[string]types.VolumeBindings) (err error) { - { - return c.withNodeLocked(ctx, nodename, func(node *types.Node) error { - - for _, container := range containers { - recycleResources(node, container) - } - planMap, total, _, err := resources.SelectNodes(rrs, map[string]*types.Node{node.Name: node}) - if err != nil { - return errors.WithStack(err) - } - if total < len(containers) { - return errors.WithStack(types.ErrInsufficientRes) - } - - var ( - rollbacks []int - originalContainers []types.Container - ) - - return utils.Txn( - ctx, - - // if: commit changes of realloc resources - func(ctx context.Context) (err error) { - for _, plan := range planMap { - plan.ApplyChangesOnNode(node, utils.Range(len(containers))...) - } - rollbacks = utils.Range(len(containers)) - for _, container := range containers { - originalContainers = append(originalContainers, *container) - } - return c.store.UpdateNodes(ctx, node) - }, - // then: update instances' resources - func(ctx context.Context) error { - rollbacks, err = c.doUpdateResourceOnInstances(ctx, ch, node, planMap, containers, hardVbsMap) - return err - }, - - // rollback: back to origin - func(ctx context.Context) error { - for _, plan := range planMap { - plan.RollbackChangesOnNode(node, rollbacks...) - } - for _, idx := range rollbacks { - preserveResources(node, &originalContainers[idx]) - } - return c.store.UpdateNodes(ctx, node) - }, - c.config.GlobalTimeout, - ) - }) - } -} - -// boundary: chan *types.ReallocResourceMessage -func (c *Calcium) doUpdateResourceOnInstances(ctx context.Context, ch chan *types.ReallocResourceMessage, node *types.Node, planMap map[types.ResourceType]resourcetypes.ResourcePlans, containers []*types.Container, hardVbsMap map[string]types.VolumeBindings) (rollbacks []int, err error) { - wg := sync.WaitGroup{} - wg.Add(len(containers)) - - for idx, container := range containers { - go func(container *types.Container, idx int) { - var e error - msg := &types.ReallocResourceMessage{ContainerID: container.ID} - defer func() { - if e != nil { - err = e - msg.Error = e - rollbacks = append(rollbacks, idx) - } - ch <- msg - wg.Done() - }() - - rsc := &types.Resources{} - for _, plan := range planMap { - if e = plan.Dispense(resourcetypes.DispenseOptions{ - Node: node, - Index: idx, - ExistingInstances: containers, - HardVolumeBindings: hardVbsMap[container.ID], - }, rsc); e != nil { - return + return utils.Txn( + ctx, + + // if update workload resources + func(ctx context.Context) (err error) { + return c.doReallocContainersOnInstance(ctx, node, plans, container) + }, + // then commit changes + func(ctx context.Context) error { + for _, plan := range plans { + plan.ApplyChangesOnNode(node, 1) } - } + return c.store.UpdateNodes(ctx, node) + }, + // no need rollback + nil, + + c.config.GlobalTimeout, + ) + }) +} - e = c.doUpdateResourceOnInstance(ctx, node, container, *rsc) - }(container, idx) +func (c *Calcium) doReallocContainersOnInstance(ctx context.Context, node *types.Node, plans []resourcetypes.ResourcePlans, container *types.Container) (err error) { + r := &types.ResourceMeta{} + for _, plan := range plans { + if r, err = plan.Dispense(resourcetypes.DispenseOptions{ + Node: node, + ExistingInstance: container, + }, r); err != nil { + return + } } - wg.Wait() - return rollbacks, errors.WithStack(err) -} - -// transaction: container meta -func (c *Calcium) doUpdateResourceOnInstance(ctx context.Context, node *types.Node, container *types.Container, rsc types.Resources) error { - originContainer := *container + originalContainer := *container return utils.Txn( ctx, - // if: update container meta - func(ctx context.Context) error { - container.CPURequest = rsc.CPURequest - container.QuotaRequest = rsc.CPUQuotaRequest - container.QuotaLimit = rsc.CPUQuotaLimit - container.MemoryRequest = rsc.MemoryRequest - container.MemoryLimit = rsc.MemoryLimit - container.SoftLimit = rsc.MemorySoftLimit - container.VolumeRequest = rsc.VolumeRequest - container.VolumePlanRequest = rsc.VolumePlanRequest - container.VolumeLimit = rsc.VolumeLimit - container.VolumePlanLimit = rsc.VolumePlanLimit - container.StorageRequest = rsc.StorageRequest - container.StorageLimit = rsc.StorageLimit - return errors.WithStack(c.store.UpdateContainer(ctx, container)) - }, - - // then: update container resources + // if: update container resources func(ctx context.Context) error { r := &enginetypes.VirtualizationResource{ - CPU: rsc.CPURequest, - Quota: rsc.CPUQuotaLimit, - NUMANode: rsc.NUMANode, - Memory: rsc.MemoryLimit, - SoftLimit: rsc.MemorySoftLimit, - Volumes: rsc.VolumeLimit.ToStringSlice(false, false), - VolumePlan: rsc.VolumePlanLimit.ToLiteral(), - VolumeChanged: rsc.VolumeChanged, - Storage: rsc.StorageLimit, + CPU: r.CPU, + Quota: r.CPUQuotaLimit, + NUMANode: r.NUMANode, + Memory: r.MemoryLimit, + Volumes: r.VolumeLimit.ToStringSlice(false, false), + VolumePlan: r.VolumePlanLimit.ToLiteral(), + VolumeChanged: r.VolumeChanged, + Storage: r.StorageLimit, } return errors.WithStack(node.Engine.VirtualizationUpdateResource(ctx, container.ID, r)) }, - // rollback: container meta + // then: update container meta func(ctx context.Context) error { - return errors.WithStack(c.store.UpdateContainer(ctx, &originContainer)) + container.CPUQuotaRequest = r.CPUQuotaRequest + container.CPUQuotaLimit = r.CPUQuotaLimit + container.CPU = r.CPU + container.MemoryRequest = r.MemoryRequest + container.MemoryLimit = r.MemoryLimit + container.VolumeRequest = r.VolumeRequest + container.VolumePlanRequest = r.VolumePlanRequest + container.VolumeLimit = r.VolumeLimit + container.VolumePlanLimit = r.VolumePlanLimit + container.StorageRequest = r.StorageRequest + container.StorageLimit = r.StorageLimit + return errors.WithStack(c.store.UpdateContainer(ctx, container)) + }, + + // rollback: container meta + func(ctx context.Context, failureByCond bool) error { + if failureByCond { + return nil + } + r := &enginetypes.VirtualizationResource{ + CPU: originalContainer.CPU, + Quota: originalContainer.CPUQuotaLimit, + NUMANode: originalContainer.NUMANode, + Memory: originalContainer.MemoryLimit, + Volumes: originalContainer.VolumeLimit.ToStringSlice(false, false), + VolumePlan: originalContainer.VolumePlanLimit.ToLiteral(), + VolumeChanged: r.VolumeChanged, + Storage: originalContainer.StorageLimit, + } + return errors.WithStack(node.Engine.VirtualizationUpdateResource(ctx, container.ID, r)) }, c.config.GlobalTimeout, ) } - -func recycleResources(node *types.Node, container *types.Container) { - node.CPU.Add(container.CPURequest) - node.SetCPUUsed(container.QuotaRequest, types.DecrUsage) - node.Volume.Add(container.VolumePlanRequest.IntoVolumeMap()) - node.SetVolumeUsed(container.VolumePlanRequest.IntoVolumeMap().Total(), types.DecrUsage) - node.StorageCap += container.StorageRequest - node.MemCap += container.MemoryRequest - if nodeID := node.GetNUMANode(container.CPURequest); nodeID != "" { - node.IncrNUMANodeMemory(nodeID, container.MemoryRequest) - } -} - -func preserveResources(node *types.Node, container *types.Container) { - node.CPU.Sub(container.CPURequest) - node.SetCPUUsed(container.QuotaRequest, types.IncrUsage) - node.Volume.Sub(container.VolumePlanRequest.IntoVolumeMap()) - node.SetVolumeUsed(container.VolumePlanRequest.IntoVolumeMap().Total(), types.IncrUsage) - node.StorageCap -= container.StorageRequest - node.MemCap -= container.MemoryRequest - if nodeID := node.GetNUMANode(container.CPURequest); nodeID != "" { - node.DecrNUMANodeMemory(nodeID, container.MemoryRequest) - } -} diff --git a/cluster/calcium/realloc_test.go b/cluster/calcium/realloc_test.go index 46f704a2c..ade8d3bc3 100644 --- a/cluster/calcium/realloc_test.go +++ b/cluster/calcium/realloc_test.go @@ -17,14 +17,15 @@ import ( "github.com/stretchr/testify/mock" ) -func newReallocOptions(ids []string, cpu float64, memory int64, vbs types.VolumeBindings, bindCPUOpt, memoryLimitOpt types.TriOptions) *types.ReallocOptions { +func newReallocOptions(id string, cpu float64, memory int64, vbs types.VolumeBindings, bindCPUOpt, memoryLimitOpt types.TriOptions) *types.ReallocOptions { return &types.ReallocOptions{ - IDs: ids, - CPU: cpu, - Memory: memory, - Volumes: vbs, - BindCPUOpt: bindCPUOpt, - MemoryLimitOpt: memoryLimitOpt, + ID: id, + CPUBindOpts: bindCPUOpt, + ResourceOpts: types.ResourceOptions{ + CPUQuotaLimit: cpu, + MemoryLimit: memory, + VolumeLimit: vbs, + }, } } @@ -42,10 +43,6 @@ func TestRealloc(t *testing.T) { engine := &enginemocks.API{} engine.On("VirtualizationInspect", mock.Anything, mock.Anything).Return(&enginetypes.VirtualizationInfo{}, nil) - pod1 := &types.Pod{ - Name: "p1", - } - node1 := &types.Node{ Name: "node1", MemCap: int64(units.GiB), @@ -58,101 +55,72 @@ func TestRealloc(t *testing.T) { Volume: types.VolumeMap{"/dir0": 100}, } - c1 := &types.Container{ - ID: "c1", - Podname: "p1", - Engine: engine, - MemoryLimit: 5 * int64(units.MiB), - MemoryRequest: 5 * int64(units.MiB), - QuotaLimit: 0.9, - QuotaRequest: 0.9, - CPURequest: types.CPUMap{"2": 90}, - Nodename: "node1", - VolumePlanRequest: types.VolumePlan{types.MustToVolumeBinding("AUTO:/data:rw:50"): types.VolumeMap{"/dir0": 50}}, - VolumeRequest: types.MustToVolumeBindings([]string{"AUTO:/data:rw:50"}), - VolumePlanLimit: types.VolumePlan{types.MustToVolumeBinding("AUTO:/data:rw:50"): types.VolumeMap{"/dir0": 50}}, - VolumeLimit: types.MustToVolumeBindings([]string{"AUTO:/data:rw:50"}), + newC1 := func(context.Context, []string) []*types.Container { + return []*types.Container{ + { + ID: "c1", + Podname: "p1", + Engine: engine, + ResourceMeta: types.ResourceMeta{ + MemoryLimit: 5 * int64(units.MiB), + MemoryRequest: 5 * int64(units.MiB), + CPUQuotaLimit: 0.9, + CPUQuotaRequest: 0.9, + CPU: types.CPUMap{"2": 90}, + VolumePlanRequest: types.VolumePlan{types.MustToVolumeBinding("AUTO:/data:rw:50"): types.VolumeMap{"/dir0": 50}}, + VolumeRequest: types.MustToVolumeBindings([]string{"AUTO:/data:rw:50"}), + VolumePlanLimit: types.VolumePlan{types.MustToVolumeBinding("AUTO:/data:rw:50"): types.VolumeMap{"/dir0": 50}}, + VolumeLimit: types.MustToVolumeBindings([]string{"AUTO:/data:rw:50"}), + }, + Nodename: "node1", + }, + } } - c2 := &types.Container{ - ID: "c2", - Podname: "p1", - Engine: engine, - MemoryRequest: 5 * int64(units.MiB), - MemoryLimit: 5 * int64(units.MiB), - QuotaLimit: 0.9, - QuotaRequest: 0.9, - Nodename: "node1", + newC2 := func(context.Context, []string) []*types.Container { + return []*types.Container{ + { + ID: "c2", + Podname: "p1", + Engine: engine, + ResourceMeta: types.ResourceMeta{ + MemoryRequest: 5 * int64(units.MiB), + MemoryLimit: 5 * int64(units.MiB), + CPUQuotaLimit: 0.9, + CPUQuotaRequest: 0.9, + }, + Nodename: "node1", + }, + } } - println(c2) - store.On("GetContainers", mock.Anything, []string{"c1"}).Return([]*types.Container{c1}, nil) + store.On("GetContainers", mock.Anything, []string{"c1"}).Return(newC1, nil) // failed by lock store.On("CreateLock", mock.Anything, mock.Anything).Return(nil, types.ErrNoETCD).Once() - ch, err := c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, -1, 2*int64(units.GiB), nil, types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i := 0 - for r := range ch { - assert.Error(t, r.Error) - i++ - } - assert.Equal(t, 1, i) + err := c.ReallocResource(ctx, newReallocOptions("c1", -1, 2*int64(units.GiB), nil, types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "ETCD must be set") store.AssertExpectations(t) - store.On("CreateLock", mock.Anything, mock.Anything).Return(lock, nil) - // failed by GetPod - store.On("GetPod", mock.Anything, mock.Anything).Return(pod1, types.ErrNoETCD).Once() - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, -1, 2*int64(units.GiB), nil, types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - assert.Error(t, r.Error) - i++ - } - assert.Equal(t, 1, i) - store.AssertExpectations(t) - store.On("GetPod", mock.Anything, mock.Anything).Return(pod1, nil) // failed by newCPU < 0 - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, -1, 2*int64(units.GiB), nil, types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - assert.Error(t, r.Error) - assert.EqualError(t, r.Error, "limit or request less than 0: bad `CPU` value") - i++ - } - assert.Equal(t, 1, i) + err = c.ReallocResource(ctx, newReallocOptions("c1", -1, 2*int64(units.GiB), nil, types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "limit or request less than 0: bad `CPU` value") store.AssertExpectations(t) // failed by GetNode store.On("GetNode", mock.Anything, "node1").Return(nil, types.ErrNoETCD).Once() - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, 0.1, 2*int64(units.GiB), nil, types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - i++ - assert.Error(t, r.Error) - assert.EqualError(t, r.Error, "ETCD must be set") - } - assert.Equal(t, 1, i) + err = c.ReallocResource(ctx, newReallocOptions("c1", 0.1, 2*int64(units.GiB), nil, types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "ETCD must be set") store.AssertExpectations(t) - // failed by no new CPU Plan + // failed by no enough mem store.On("GetNode", mock.Anything, "node1").Return(node1, nil) simpleMockScheduler := &schedulermocks.Scheduler{} scheduler.InitSchedulerV1(simpleMockScheduler) c.scheduler = simpleMockScheduler simpleMockScheduler.On("SelectCPUNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, nil, 0, types.ErrInsufficientMEM).Once() - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - assert.Error(t, r.Error) - assert.EqualError(t, r.Error, "cannot alloc a plan, not enough memory") - i++ - } - assert.Equal(t, 1, i) + err = c.ReallocResource(ctx, newReallocOptions("c1", 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "cannot alloc a plan, not enough memory") store.AssertExpectations(t) simpleMockScheduler.AssertExpectations(t) @@ -162,16 +130,9 @@ func TestRealloc(t *testing.T) { nodeVolumePlans := map[string][]types.VolumePlan{ "node1": {{types.MustToVolumeBinding("AUTO:/data:rw:50"): types.VolumeMap{"/dir0": 50}}}, } - simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, types.MustToVolumeBindings([]string{"AUTO:/data:rw:50"})).Return(nil, nodeVolumePlans, 1, nil) - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - i++ - assert.Error(t, r.Error) - assert.EqualError(t, r.Error, "not enough resource") - } - assert.Equal(t, 1, i) + simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, types.MustToVolumeBindings([]string{"AUTO:/data:rw:50"})).Return(nil, nodeVolumePlans, 1, nil).Once() + err = c.ReallocResource(ctx, newReallocOptions("c1", 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "not enough resource") simpleMockScheduler.AssertExpectations(t) store.AssertExpectations(t) @@ -182,13 +143,10 @@ func TestRealloc(t *testing.T) { {"2": 100}, }, } - simpleMockScheduler.On("SelectCPUNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, nodeCPUPlans, 2, nil).Once() simpleMockScheduler.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, 2, nil).Once() - simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, types.VolumeBindings{}).Return(nil, nil, 100, nil) + simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything).Return(nil, nil, 100, nil).Once() // failed by apply resource - engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(types.ErrBadContainerID).Twice() - // update node failed - store.On("UpdateNodes", mock.Anything, mock.Anything).Return(types.ErrNoETCD).Times(4) + engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(types.ErrBadContainerID).Once() // reset node node1 = &types.Node{ Name: "node1", @@ -197,36 +155,21 @@ func TestRealloc(t *testing.T) { Engine: engine, Endpoint: "http://1.1.1.1:1", } - store.On("GetContainers", mock.Anything, []string{"c1", "c2"}).Return([]*types.Container{c1, c2}, nil) - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1", "c2"}, 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - assert.Error(t, r.Error) - i++ - } - // check node resource as usual - assert.Equal(t, 2, i) + store.On("GetContainers", mock.Anything, []string{"c2"}).Return(newC2, nil) + err = c.ReallocResource(ctx, newReallocOptions("c2", 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "container ID must be length of 64") assert.Equal(t, node1.CPU["2"], int64(10)) assert.Equal(t, node1.MemCap, int64(units.GiB)) simpleMockScheduler.AssertExpectations(t) store.AssertExpectations(t) + // failed by update container simpleMockScheduler.On("SelectCPUNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, nodeCPUPlans, 2, nil).Once() - simpleMockScheduler.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, 2, nil).Once() + simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything).Return(nil, nil, 100, nil).Once() engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(nil) - store.On("UpdateNodes", mock.Anything, mock.Anything).Return(nil) - - // failed by update container - store.On("UpdateContainer", mock.Anything, mock.Anything).Return(types.ErrBadContainerID).Times(4) - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1", "c2"}, 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - i++ - assert.Error(t, r.Error) - } - assert.GreaterOrEqual(t, i, 2) + store.On("UpdateContainer", mock.Anything, mock.Anything).Return(types.ErrBadContainerID).Times(1) + err = c.ReallocResource(ctx, newReallocOptions("c1", 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "container ID must be length of 64") simpleMockScheduler.AssertExpectations(t) store.AssertExpectations(t) @@ -240,45 +183,20 @@ func TestRealloc(t *testing.T) { }, } simpleMockScheduler.On("SelectCPUNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, nodeCPUPlans, 2, nil).Once() - simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, types.MustToVolumeBindings([]string{"AUTO:/data:rw:100"})).Return(nil, nodeVolumePlans, 4, nil).Once() - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, 0.1, int64(units.MiB), types.MustToVolumeBindings([]string{"AUTO:/data:rw:50"}), types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - i++ - assert.Error(t, r.Error) - } - assert.Equal(t, 2, i) + simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything).Return(nil, nodeVolumePlans, 4, nil).Once() + err = c.ReallocResource(ctx, newReallocOptions("c1", 0.1, int64(units.MiB), types.MustToVolumeBindings([]string{"AUTO:/data:rw:50"}), types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "incompatible volume plans: cannot alloc a plan, not enough volume") simpleMockScheduler.AssertExpectations(t) store.AssertExpectations(t) // failed by volume schedule error simpleMockScheduler.On("SelectCPUNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, nodeCPUPlans, 2, nil).Once() simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything).Return(nil, nil, 0, types.ErrInsufficientVolume).Once() - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, 0.1, int64(units.MiB), types.MustToVolumeBindings([]string{"AUTO:/data:rw:1"}), types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - assert.Error(t, r.Error) - i++ - } - assert.Equal(t, i, 1) + err = c.ReallocResource(ctx, newReallocOptions("c1", 0.1, int64(units.MiB), types.MustToVolumeBindings([]string{"AUTO:/data:rw:1"}), types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "cannot alloc a plan, not enough volume") simpleMockScheduler.AssertExpectations(t) store.AssertExpectations(t) - // failed due to re-volume plan less then container number - simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything).Return(nil, nodeVolumePlans, 0, nil).Once() - simpleMockScheduler.On("SelectCPUNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, nodeCPUPlans, 2, nil).Once() - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, 0.1, int64(units.MiB), types.MustToVolumeBindings([]string{"AUTO:/data:rw:1"}), types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - assert.Error(t, r.Error) - i++ - } - assert.Equal(t, 1, i) - simpleMockScheduler.AssertExpectations(t) - // good to go // rest everything node2 := &types.Node{ @@ -294,42 +212,31 @@ func TestRealloc(t *testing.T) { VolumeUsed: int64(300), } c3 := &types.Container{ - ID: "c3", - Podname: "p1", - Engine: engine, - MemoryLimit: 5 * int64(units.MiB), - MemoryRequest: 5 * int64(units.MiB), - QuotaLimit: 0.9, - QuotaRequest: 0.9, - CPURequest: types.CPUMap{"2": 90}, - VolumeRequest: types.MustToVolumeBindings([]string{"AUTO:/data0:rw:100", "AUTO:/data1:rw:200"}), - VolumePlanRequest: types.VolumePlan{ - types.MustToVolumeBinding("AUTO:/data0:rw:100"): types.VolumeMap{"/dir0": 100}, - types.MustToVolumeBinding("AUTO:/data1:rw:200"): types.VolumeMap{"/dir1": 200}, - }, - VolumeLimit: types.MustToVolumeBindings([]string{"AUTO:/data0:rw:100", "AUTO:/data1:rw:200"}), - VolumePlanLimit: types.VolumePlan{ - types.MustToVolumeBinding("AUTO:/data0:rw:100"): types.VolumeMap{"/dir0": 100}, - types.MustToVolumeBinding("AUTO:/data1:rw:200"): types.VolumeMap{"/dir1": 200}, + ID: "c3", + Podname: "p1", + Engine: engine, + ResourceMeta: types.ResourceMeta{ + MemoryLimit: 5 * int64(units.MiB), + MemoryRequest: 5 * int64(units.MiB), + CPUQuotaLimit: 0.9, + CPUQuotaRequest: 0.9, + CPU: types.CPUMap{"2": 90}, + VolumeRequest: types.MustToVolumeBindings([]string{"AUTO:/data0:rw:100", "AUTO:/data1:rw:200"}), + VolumePlanRequest: types.VolumePlan{ + types.MustToVolumeBinding("AUTO:/data0:rw:100"): types.VolumeMap{"/dir0": 100}, + types.MustToVolumeBinding("AUTO:/data1:rw:200"): types.VolumeMap{"/dir1": 200}, + }, + VolumeLimit: types.MustToVolumeBindings([]string{"AUTO:/data0:rw:100", "AUTO:/data1:rw:200"}), + VolumePlanLimit: types.VolumePlan{ + types.MustToVolumeBinding("AUTO:/data0:rw:100"): types.VolumeMap{"/dir0": 100}, + types.MustToVolumeBinding("AUTO:/data1:rw:200"): types.VolumeMap{"/dir1": 200}, + }, }, Nodename: "node2", } - c4 := &types.Container{ - ID: "c4", - Podname: "p1", - Engine: engine, - MemoryRequest: 5 * int64(units.MiB), - MemoryLimit: 5 * int64(units.MiB), - QuotaRequest: 0.9, - QuotaLimit: 0.9, - VolumeRequest: types.MustToVolumeBindings([]string{"/tmp:/tmp", "/var/log:/var/log:rw:300"}), - VolumeLimit: types.MustToVolumeBindings([]string{"/tmp:/tmp", "/var/log:/var/log:rw:300"}), - Nodename: "node2", - } nodeCPUPlans = map[string][]types.CPUMap{ node2.Name: { {"3": 100}, - {"2": 100}, }, } nodeVolumePlans = map[string][]types.VolumePlan{ @@ -345,225 +252,19 @@ func TestRealloc(t *testing.T) { }, } simpleMockScheduler.On("SelectCPUNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, nodeCPUPlans, 2, nil) - simpleMockScheduler.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, 2, nil).Once() - simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, types.MustToVolumeBindings([]string{"AUTO:/data0:rw:50", "AUTO:/data1:rw:200"})).Return(nil, nodeVolumePlans, 2, nil) + simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything).Return(nil, nodeVolumePlans, 2, nil) store.On("GetNode", mock.Anything, "node2").Return(node2, nil) - store.On("GetContainers", mock.Anything, []string{"c3", "c4"}).Return([]*types.Container{c3, c4}, nil) - store.On("UpdateContainer", mock.Anything, mock.Anything).Return(types.ErrBadContainerID).Times(4) - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c3", "c4"}, 0.1, 2*int64(units.MiB), types.MustToVolumeBindings([]string{"AUTO:/data0:rw:-50"}), types.TriKeep, types.TriKeep)) - assert.NoError(t, err) - i = 0 - for r := range ch { - i++ - assert.Error(t, r.Error) - } - assert.GreaterOrEqual(t, i, 2) + store.On("GetContainers", mock.Anything, []string{"c3"}).Return([]*types.Container{c3}, nil) + store.On("UpdateContainer", mock.Anything, mock.Anything).Return(types.ErrBadContainerID).Times(1) + err = c.ReallocResource(ctx, newReallocOptions("c3", 0.1, 2*int64(units.MiB), types.MustToVolumeBindings([]string{"AUTO:/data0:rw:-50"}), types.TriKeep, types.TriKeep)) + assert.EqualError(t, err, "container ID must be length of 64") assert.Equal(t, node2.CPU["3"], int64(100)) - assert.Equal(t, node2.CPU["2"], int64(10)) - assert.Equal(t, node2.MemCap, int64(units.GiB)) - assert.Equal(t, node2.Volume, types.VolumeMap{"/dir0": 200, "/dir1": 200, "/dir2": 200}) - assert.Equal(t, node2.VolumeUsed, int64(300)) - store.AssertExpectations(t) - simpleMockScheduler.AssertExpectations(t) -} - -func _TestReallocVolume(t *testing.T) { - c := NewTestCluster() - store := &storemocks.Store{} - c.store = store - - simpleMockScheduler := &schedulermocks.Scheduler{} - c.scheduler = simpleMockScheduler - scheduler.InitSchedulerV1(simpleMockScheduler) - engine := &enginemocks.API{} - - node1 := &types.Node{ - Name: "node1", - Volume: types.VolumeMap{"/data": 1000, "/data1": 1000, "/data2": 1000, "/data3": 1000}, - InitVolume: types.VolumeMap{"/data": 2000, "/data1": 2000, "/data2": 2000, "/data3": 2000}, - Engine: engine, - } - - c1 := &types.Container{ - ID: "c1", - Engine: engine, - Podname: "p1", - Nodename: "node1", - VolumeRequest: types.MustToVolumeBindings([]string{"AUTO:/data:rw:0", "AUTO:/data1:rw:100", "AUTO:/data2:rw:0", "AUTO:/data3:rw:600"}), - VolumeLimit: types.MustToVolumeBindings([]string{"AUTO:/data:rw:0", "AUTO:/data1:rw:100", "AUTO:/data2:rw:0", "AUTO:/data3:rw:600"}), - VolumePlanRequest: types.VolumePlan{ - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:100"): types.VolumeMap{"/dir0": 100}, - types.MustToVolumeBinding("AUTO:/data2:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data3:rw:600"): types.VolumeMap{"/dir0": 600}, - }, - VolumePlanLimit: types.VolumePlan{ - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:100"): types.VolumeMap{"/dir0": 100}, - types.MustToVolumeBinding("AUTO:/data2:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data3:rw:600"): types.VolumeMap{"/dir0": 600}, - }, - } - - pod1 := &types.Pod{Name: "p1"} - - newVbs := types.MustToVolumeBindings([]string{ - "AUTO:/data:rw:0", - "AUTO:/data1:rw:-100", - "AUTO:/data2:ro:110", - "AUTO:/data3:rw:-580", - }) - - // test 1: incompatible - - newPlans := map[string][]types.VolumePlan{ - "node1": { - { - *newVbs[0]: types.VolumeMap{"/dir1": 0}, - *newVbs[1]: types.VolumeMap{"/dir1": 0}, - *newVbs[2]: types.VolumeMap{"/dir0": 110}, - *newVbs[3]: types.VolumeMap{"/dir1": 20}, - }, - }, - } - - ctx := context.Background() - simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything).Return(nil, newPlans, 1, nil).Once() - simpleMockScheduler.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, 100, nil) - simpleMockScheduler.On("SelectStorageNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil, 100, nil) - lock := &lockmocks.DistributedLock{} - lock.On("Lock", mock.Anything).Return(nil) - lock.On("Unlock", mock.Anything).Return(nil) - store.On("CreateLock", mock.Anything, mock.Anything).Return(lock, nil) - store.On("GetPod", mock.Anything, mock.Anything).Return(pod1, nil) - store.On("GetContainers", mock.Anything, []string{"c1"}).Return([]*types.Container{c1}, nil) - store.On("GetNode", mock.Anything, "node1").Return(node1, nil) - store.On("UpdateNodes", mock.Anything, mock.Anything).Return(nil) - ch, err := c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, 0, 0, newVbs, types.TriKeep, types.TriKeep)) - assert.Nil(t, err) - i := 0 - for m := range ch { - i++ - assert.Error(t, m.Error) - assert.EqualError(t, m.Error, "incompatible volume plans: cannot alloc a plan, not enough volume") - } - assert.GreaterOrEqual(t, i, 1) - simpleMockScheduler.AssertExpectations(t) + assert.Equal(t, node2.CPU["2"], int64(100)) + assert.Equal(t, node2.MemCap, int64(units.GiB)+5*int64(units.MiB)) + assert.Equal(t, node2.Volume, types.VolumeMap{"/dir0": 300, "/dir1": 400, "/dir2": 200}) + assert.Equal(t, node2.VolumeUsed, int64(0)) store.AssertExpectations(t) - - // test 2: modify unlimited volume map for compatible requirement - - newPlans = map[string][]types.VolumePlan{ - "node1": { - { - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data2:rw:110"): types.VolumeMap{"/dir1": 110}, - types.MustToVolumeBinding("AUTO:/data3:rw:20"): types.VolumeMap{"/dir1": 20}, - }, - { - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data2:rw:110"): types.VolumeMap{"/dir0": 110}, - types.MustToVolumeBinding("AUTO:/data3:rw:20"): types.VolumeMap{"/dir0": 20}, - }, - }, - } - - simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything).Return(nil, newPlans, 1, nil).Once() - store.On("UpdateContainer", mock.Anything, mock.Anything).Return(nil) - engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(nil) - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1"}, 0, 0, newVbs, types.TriKeep, types.TriKeep)) - assert.Nil(t, err) - for m := range ch { - assert.Nil(t, m.Error) - } - assert.EqualValues(t, 0, c1.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data:rw:0")]["/dir0"]) - assert.EqualValues(t, 0, c1.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data1:rw:0")]["/dir0"]) - assert.EqualValues(t, 110, c1.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data2:rw:110")]["/dir0"]) - assert.EqualValues(t, 20, c1.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data3:rw:20")]["/dir0"]) simpleMockScheduler.AssertExpectations(t) - - // test 3: multiple containers search compatible respective plans - - newPlans = map[string][]types.VolumePlan{ - "node1": { - { - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data2:rw:110"): types.VolumeMap{"/dir0": 110}, - types.MustToVolumeBinding("AUTO:/data3:rw:20"): types.VolumeMap{"/dir1": 20}, - }, - { - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data2:rw:110"): types.VolumeMap{"/dir0": 110}, - types.MustToVolumeBinding("AUTO:/data3:rw:20"): types.VolumeMap{"/dir0": 20}, - }, - { - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data2:rw:110"): types.VolumeMap{"/dir1": 110}, - types.MustToVolumeBinding("AUTO:/data3:rw:20"): types.VolumeMap{"/dir0": 20}, - }, - }, - } - - c1.VolumeRequest = types.MustToVolumeBindings([]string{"AUTO:/data:rw:0", "AUTO:/data1:rw:100", "AUTO:/data2:rw:0", "AUTO:/data3:rw:600"}) - c1.VolumeLimit = types.MustToVolumeBindings([]string{"AUTO:/data:rw:0", "AUTO:/data1:rw:100", "AUTO:/data2:rw:0", "AUTO:/data3:rw:600"}) - c1.VolumePlanLimit = types.VolumePlan{ - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:100"): types.VolumeMap{"/dir0": 100}, - types.MustToVolumeBinding("AUTO:/data2:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data3:rw:600"): types.VolumeMap{"/dir0": 600}, - } - c1.VolumePlanRequest = types.VolumePlan{ - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:100"): types.VolumeMap{"/dir0": 100}, - types.MustToVolumeBinding("AUTO:/data2:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data3:rw:600"): types.VolumeMap{"/dir0": 600}, - } - - c2 := &types.Container{ - ID: "c2", - Engine: engine, - Podname: "p1", - Nodename: "node1", - VolumeRequest: types.MustToVolumeBindings([]string{"AUTO:/data:rw:0", "AUTO:/data1:rw:100", "AUTO:/data2:rw:0", "AUTO:/data3:rw:600"}), - VolumeLimit: types.MustToVolumeBindings([]string{"AUTO:/data:rw:0", "AUTO:/data1:rw:100", "AUTO:/data2:rw:0", "AUTO:/data3:rw:600"}), - VolumePlanRequest: types.VolumePlan{ - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:100"): types.VolumeMap{"/dir1": 100}, - types.MustToVolumeBinding("AUTO:/data2:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data3:rw:600"): types.VolumeMap{"/dir0": 600}, - }, - VolumePlanLimit: types.VolumePlan{ - types.MustToVolumeBinding("AUTO:/data:rw:0"): types.VolumeMap{"/dir0": 0}, - types.MustToVolumeBinding("AUTO:/data1:rw:100"): types.VolumeMap{"/dir1": 100}, - types.MustToVolumeBinding("AUTO:/data2:rw:0"): types.VolumeMap{"/dir1": 0}, - types.MustToVolumeBinding("AUTO:/data3:rw:600"): types.VolumeMap{"/dir0": 600}, - }, - } - - simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything).Return(nil, newPlans, 3, nil).Once() - store.On("GetContainers", mock.Anything, []string{"c1", "c2"}).Return([]*types.Container{c1, c2}, nil) - - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c1", "c2"}, 0, 0, newVbs, types.TriKeep, types.TriKeep)) - assert.Nil(t, err) - i = 0 - for m := range ch { - i++ - assert.Nil(t, m.Error) - } - assert.Equal(t, 2, i) - assert.EqualValues(t, 0, c1.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data:rw:0")]["/dir0"]) - assert.EqualValues(t, 0, c1.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data1:rw:0")]["/dir0"]) - assert.EqualValues(t, 110, c1.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data2:rw:110")]["/dir0"]) - assert.EqualValues(t, 20, c1.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data3:rw:20")]["/dir0"]) - assert.EqualValues(t, 0, c2.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data:rw:0")]["/dir0"]) - assert.EqualValues(t, 0, c2.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data1:rw:0")]["/dir1"]) - assert.EqualValues(t, 110, c2.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data2:rw:110")]["/dir1"]) - assert.EqualValues(t, 20, c2.VolumePlanRequest[types.MustToVolumeBinding("AUTO:/data3:rw:20")]["/dir0"]) } func TestReallocBindCpu(t *testing.T) { @@ -619,25 +320,29 @@ func TestReallocBindCpu(t *testing.T) { VolumeUsed: int64(300), } c5 := &types.Container{ - ID: "c5", - Podname: "p1", - Engine: engine, - MemoryRequest: 5 * int64(units.MiB), - MemoryLimit: 5 * int64(units.MiB), - QuotaRequest: 0.9, - QuotaLimit: 0.9, - CPURequest: types.CPUMap{"2": 90}, - Nodename: "node3", + ID: "c5", + Podname: "p1", + Engine: engine, + ResourceMeta: types.ResourceMeta{ + MemoryRequest: 5 * int64(units.MiB), + MemoryLimit: 5 * int64(units.MiB), + CPUQuotaRequest: 0.9, + CPUQuotaLimit: 0.9, + CPU: types.CPUMap{"2": 90}, + }, + Nodename: "node3", } c6 := &types.Container{ - ID: "c6", - Podname: "p1", - Engine: engine, - MemoryRequest: 5 * int64(units.MiB), - MemoryLimit: 5 * int64(units.MiB), - QuotaRequest: 0.9, - QuotaLimit: 0.9, - Nodename: "node3", + ID: "c6", + Podname: "p1", + Engine: engine, + ResourceMeta: types.ResourceMeta{ + MemoryRequest: 5 * int64(units.MiB), + MemoryLimit: 5 * int64(units.MiB), + CPUQuotaRequest: 0.9, + CPUQuotaLimit: 0.9, + }, + Nodename: "node3", } store.On("GetNode", mock.Anything, "node3").Return(node3, nil) @@ -647,33 +352,23 @@ func TestReallocBindCpu(t *testing.T) { engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(nil) store.On("UpdateNodes", mock.Anything, mock.Anything).Return(nil) store.On("UpdateContainer", mock.Anything, mock.Anything).Return(nil) - ch, err := c.ReallocResource(ctx, newReallocOptions([]string{"c5"}, 0.1, 2*int64(units.MiB), nil, types.TriFalse, types.TriKeep)) - for r := range ch { - assert.NoError(t, r.Error) - } + err := c.ReallocResource(ctx, newReallocOptions("c5", 0.1, 2*int64(units.MiB), nil, types.TriFalse, types.TriKeep)) + assert.NoError(t, err) - assert.Equal(t, 0, len(c5.CPURequest)) + assert.Equal(t, 0, len(c5.ResourceMeta.CPU)) + + err = c.ReallocResource(ctx, newReallocOptions("c6", 0.1, 2*int64(units.MiB), nil, types.TriTrue, types.TriKeep)) - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c6"}, 0.1, 2*int64(units.MiB), nil, types.TriTrue, types.TriKeep)) - for r := range ch { - assert.NoError(t, r.Error) - } assert.NoError(t, err) - assert.NotEmpty(t, c6.CPURequest) + assert.NotEmpty(t, c6.ResourceMeta.CPU) node3.CPU = types.CPUMap{"0": 10, "1": 70, "2": 100, "3": 100} - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c6", "c5"}, -0.1, 2*int64(units.MiB), nil, types.TriTrue, types.TriKeep)) - for r := range ch { - assert.NoError(t, r.Error) - } + err = c.ReallocResource(ctx, newReallocOptions("c5", -0.1, 2*int64(units.MiB), nil, types.TriTrue, types.TriKeep)) + assert.NoError(t, err) - assert.NotEmpty(t, c6.CPURequest) - assert.NotEmpty(t, c5.CPURequest) - ch, err = c.ReallocResource(ctx, newReallocOptions([]string{"c6", "c5"}, -0.1, 2*int64(units.MiB), nil, types.TriFalse, types.TriKeep)) - for r := range ch { - assert.NoError(t, r.Error) - } + assert.NotEmpty(t, c5.ResourceMeta.CPU) + err = c.ReallocResource(ctx, newReallocOptions("c6", -0.1, 2*int64(units.MiB), nil, types.TriFalse, types.TriKeep)) + assert.NoError(t, err) - assert.Equal(t, 0, len(c5.CPURequest)) - assert.Equal(t, 0, len(c6.CPURequest)) + assert.Equal(t, 0, len(c6.ResourceMeta.CPU)) } diff --git a/cluster/calcium/remove.go b/cluster/calcium/remove.go index 75cdf8e80..442ed78bc 100644 --- a/cluster/calcium/remove.go +++ b/cluster/calcium/remove.go @@ -40,7 +40,7 @@ func (c *Calcium) RemoveContainer(ctx context.Context, IDs []string, force bool, // then func(ctx context.Context) error { log.Infof("[RemoveContainer] Container %s removed", container.ID) - return c.store.UpdateNodeResource(ctx, node, container.CPURequest, container.QuotaRequest, container.MemoryRequest, container.StorageRequest, container.VolumePlanRequest.IntoVolumeMap(), store.ActionIncr) + return c.store.UpdateNodeResource(ctx, node, &container.ResourceMeta, store.ActionIncr) }, // rollback nil, diff --git a/cluster/calcium/remove_test.go b/cluster/calcium/remove_test.go index 38ccc5d02..37ddb977e 100644 --- a/cluster/calcium/remove_test.go +++ b/cluster/calcium/remove_test.go @@ -57,7 +57,7 @@ func TestRemoveContainer(t *testing.T) { engine.On("VirtualizationRemove", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) store.On("GetContainers", mock.Anything, mock.Anything).Return([]*types.Container{container}, nil) store.On("RemoveContainer", mock.Anything, mock.Anything).Return(nil) - store.On("UpdateNodeResource", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) + store.On("UpdateNodeResource", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) // success ch, err = c.RemoveContainer(ctx, []string{"xx"}, false, 0) assert.NoError(t, err) diff --git a/cluster/calcium/replace.go b/cluster/calcium/replace.go index 8f264571a..65fdd7527 100644 --- a/cluster/calcium/replace.go +++ b/cluster/calcium/replace.go @@ -57,7 +57,17 @@ func (c *Calcium) ReplaceContainer(ctx context.Context, opts *types.ReplaceOptio } // 使用复制之后的配置 // 停老的,起新的 - replaceOpts.SoftLimit = container.SoftLimit + replaceOpts.ResourceOpts = types.ResourceOptions{ + CPUQuotaRequest: container.CPUQuotaRequest, + CPUQuotaLimit: container.CPUQuotaLimit, + CPUBind: len(container.CPU) > 0, + MemoryRequest: container.MemoryRequest, + MemoryLimit: container.MemoryLimit, + StorageRequest: container.StorageRequest, + StorageLimit: container.StorageLimit, + VolumeRequest: container.VolumeRequest, + VolumeLimit: container.VolumeLimit, + } // 覆盖 podname 如果做全量更新的话 replaceOpts.Podname = container.Podname // 覆盖 Volumes @@ -128,14 +138,14 @@ func (c *Calcium) doReplaceContainer( } createMessage := &types.CreateContainerMessage{ - Resources: types.Resources{ + ResourceMeta: types.ResourceMeta{ MemoryRequest: container.MemoryRequest, MemoryLimit: container.MemoryLimit, StorageRequest: container.StorageRequest, StorageLimit: container.StorageLimit, - CPUQuotaRequest: container.QuotaRequest, - CPUQuotaLimit: container.QuotaLimit, - CPURequest: container.CPURequest, + CPUQuotaRequest: container.CPUQuotaRequest, + CPUQuotaLimit: container.CPUQuotaLimit, + CPU: container.CPU, VolumeRequest: container.VolumeRequest, VolumePlanRequest: container.VolumePlanRequest, VolumeLimit: container.VolumeLimit, @@ -173,7 +183,7 @@ func (c *Calcium) doReplaceContainer( ) }, // rollback - func(ctx context.Context) (err error) { + func(ctx context.Context, _ bool) (err error) { messages, err := c.doStartContainer(ctx, container, opts.IgnoreHook) if err != nil { log.Errorf("[replaceAndRemove] Old container %s restart failed %v", container.ID, err) diff --git a/cluster/calcium/replace_test.go b/cluster/calcium/replace_test.go index 8b5467a23..5f4764f0c 100644 --- a/cluster/calcium/replace_test.go +++ b/cluster/calcium/replace_test.go @@ -134,7 +134,7 @@ func TestReplaceContainer(t *testing.T) { // failed by VirtualizationCreate engine.On("VirtualizationCreate", mock.Anything, mock.Anything).Return(nil, types.ErrCannotGetEngine).Once() engine.On("VirtualizationStart", mock.Anything, mock.Anything).Return(types.ErrCannotGetEngine).Once() - store.On("UpdateNodeResource", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Once() + store.On("UpdateNodeResource", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Once() engine.On("VirtualizationRemove", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Once() store.On("RemoveContainer", mock.Anything, mock.Anything).Return(nil).Once() ch, err = c.ReplaceContainer(ctx, opts) diff --git a/cluster/calcium/resource.go b/cluster/calcium/resource.go index 7accf9a70..a29a4ef0f 100644 --- a/cluster/calcium/resource.go +++ b/cluster/calcium/resource.go @@ -78,10 +78,10 @@ func (c *Calcium) doGetNodeResource(ctx context.Context, nodename string, fix bo storage := int64(0) cpumap := types.CPUMap{} for _, container := range containers { - cpus = utils.Round(cpus + container.QuotaRequest) + cpus = utils.Round(cpus + container.CPUQuotaRequest) memory += container.MemoryRequest storage += container.StorageRequest - cpumap.Add(container.CPURequest) + cpumap.Add(container.CPU) } nr.CPUPercent = cpus / float64(len(node.InitCPU)) nr.MemoryPercent = float64(memory) / float64(node.InitMemCap) @@ -156,25 +156,25 @@ func (c *Calcium) doFixDiffResource(ctx context.Context, node *types.Node, cpus ) } -func (c *Calcium) doAllocResource(ctx context.Context, nodeMap map[string]*types.Node, opts *types.DeployOptions) (map[types.ResourceType]resourcetypes.ResourcePlans, map[string]int, error) { +func (c *Calcium) doAllocResource(ctx context.Context, nodeMap map[string]*types.Node, opts *types.DeployOptions) ([]resourcetypes.ResourcePlans, map[string]int, error) { if len(nodeMap) == 0 { return nil, nil, errors.WithStack(types.ErrInsufficientNodes) } - apps, err := resources.NewResourceRequirements(opts.RawResourceOptions) + resourceRequests, err := resources.MakeRequests(opts.ResourceOpts) if err != nil { return nil, nil, errors.WithStack(err) } // select available nodes - planMap, total, scheduleTypes, err := resources.SelectNodes(apps, nodeMap) + scheduleTypes, total, plans, err := resources.SelectNodesByResourceRequests(resourceRequests, nodeMap) if err != nil { return nil, nil, errors.WithStack(err) } - log.Debugf("[Calcium.doAllocResource] planMap: %+v, total: %v, type: %+v", planMap, total, scheduleTypes) + log.Debugf("[Calcium.doAllocResource] plans: %+v, total: %v, type: %+v", plans, total, scheduleTypes) // deploy strategy - strategyInfos := strategy.NewInfos(apps, nodeMap, planMap) + strategyInfos := strategy.NewInfos(resourceRequests, nodeMap, plans) if err := c.store.MakeDeployStatus(ctx, opts, strategyInfos); err != nil { return nil, nil, errors.WithStack(err) } @@ -183,5 +183,5 @@ func (c *Calcium) doAllocResource(ctx context.Context, nodeMap map[string]*types return nil, nil, errors.WithStack(err) } log.Infof("[Calium.doAllocResource] deployMap: %+v", deployMap) - return planMap, deployMap, nil + return plans, deployMap, nil } diff --git a/cluster/calcium/resource_test.go b/cluster/calcium/resource_test.go index dc49d27d5..bcefd17ca 100644 --- a/cluster/calcium/resource_test.go +++ b/cluster/calcium/resource_test.go @@ -51,20 +51,24 @@ func TestPodResource(t *testing.T) { assert.Error(t, err) containers := []*types.Container{ { - MemoryRequest: 1, - MemoryLimit: 1, - CPURequest: types.CPUMap{"0": 100, "1": 30}, - QuotaRequest: 1.3, - QuotaLimit: 1.3, + ResourceMeta: types.ResourceMeta{ + MemoryRequest: 1, + MemoryLimit: 1, + CPU: types.CPUMap{"0": 100, "1": 30}, + CPUQuotaRequest: 1.3, + CPUQuotaLimit: 1.3, + }, }, { - MemoryLimit: 2, - MemoryRequest: 2, - CPURequest: types.CPUMap{"1": 50}, - QuotaRequest: 0.5, - QuotaLimit: 0.5, - StorageRequest: 1, - StorageLimit: 1, + ResourceMeta: types.ResourceMeta{ + MemoryLimit: 2, + MemoryRequest: 2, + CPU: types.CPUMap{"1": 50}, + CPUQuotaRequest: 0.5, + CPUQuotaLimit: 0.5, + StorageRequest: 1, + StorageLimit: 1, + }, }, } store.On("ListNodeContainers", mock.Anything, mock.Anything, mock.Anything).Return(containers, nil) @@ -118,18 +122,22 @@ func TestNodeResource(t *testing.T) { assert.Error(t, err) containers := []*types.Container{ { - MemoryRequest: 1, - MemoryLimit: 1, - CPURequest: types.CPUMap{"0": 100, "1": 30}, - QuotaRequest: 1.3, - QuotaLimit: 1.3, + ResourceMeta: types.ResourceMeta{ + MemoryRequest: 1, + MemoryLimit: 1, + CPU: types.CPUMap{"0": 100, "1": 30}, + CPUQuotaRequest: 1.3, + CPUQuotaLimit: 1.3, + }, }, { - MemoryRequest: 2, - MemoryLimit: 2, - CPURequest: types.CPUMap{"1": 50}, - QuotaRequest: 0.5, - QuotaLimit: 0.5, + ResourceMeta: types.ResourceMeta{ + MemoryRequest: 2, + MemoryLimit: 2, + CPU: types.CPUMap{"1": 50}, + CPUQuotaRequest: 0.5, + CPUQuotaLimit: 0.5, + }, }, } store.On("ListNodeContainers", mock.Anything, mock.Anything, mock.Anything).Return(containers, nil) @@ -222,7 +230,7 @@ func TestAllocResource(t *testing.T) { }() // success - opts.RawResourceOptions = types.RawResourceOptions{CPULimit: 1, MemoryLimit: 1, StorageLimit: 1} + opts.ResourceOpts = types.ResourceOptions{CPUQuotaLimit: 1, MemoryLimit: 1, StorageLimit: 1} _, _, err := c.doAllocResource(ctx, nodeMap, opts) assert.NoError(t, err) } @@ -235,7 +243,7 @@ func testAllocFailedAsMakeDeployStatusError(t *testing.T, c *Calcium, opts *type } func testAllocFailedAsInsufficientMemory(t *testing.T, c *Calcium, opts *types.DeployOptions, nodeMap map[string]*types.Node) { - opts.RawResourceOptions = types.RawResourceOptions{CPULimit: 1, MemoryLimit: 1} + opts.ResourceOpts = types.ResourceOptions{CPUQuotaLimit: 1, MemoryLimit: 1} _, _, err := c.doAllocResource(context.Background(), nodeMap, opts) assert.Error(t, err) } diff --git a/cluster/cluster.go b/cluster/cluster.go index 4c8d352a9..b486f2dfc 100644 --- a/cluster/cluster.go +++ b/cluster/cluster.go @@ -79,7 +79,7 @@ type Cluster interface { DissociateContainer(ctx context.Context, IDs []string) (chan *types.DissociateContainerMessage, error) ControlContainer(ctx context.Context, IDs []string, t string, force bool) (chan *types.ControlContainerMessage, error) ExecuteContainer(ctx context.Context, opts *types.ExecuteContainerOptions, inCh <-chan []byte) chan *types.AttachContainerMessage - ReallocResource(ctx context.Context, opts *types.ReallocOptions) (chan *types.ReallocResourceMessage, error) + ReallocResource(ctx context.Context, opts *types.ReallocOptions) error LogStream(ctx context.Context, opts *types.LogStreamOptions) (chan *types.LogStreamMessage, error) RunAndWait(ctx context.Context, opts *types.DeployOptions, inCh <-chan []byte) (<-chan *types.AttachContainerMessage, error) // finalizer diff --git a/cluster/mocks/Cluster.go b/cluster/mocks/Cluster.go index c7c4cc2b9..b75020cb4 100644 --- a/cluster/mocks/Cluster.go +++ b/cluster/mocks/Cluster.go @@ -1,11 +1,15 @@ -// Code generated by mockery v1.0.0. DO NOT EDIT. +// Code generated by mockery v2.0.0-alpha.2. DO NOT EDIT. package mocks -import context "context" -import enginetypes "github.com/projecteru2/core/engine/types" -import mock "github.com/stretchr/testify/mock" -import types "github.com/projecteru2/core/types" +import ( + context "context" + + enginetypes "github.com/projecteru2/core/engine/types" + mock "github.com/stretchr/testify/mock" + + types "github.com/projecteru2/core/types" +) // Cluster is an autogenerated mock type for the Cluster type type Cluster struct { @@ -570,26 +574,17 @@ func (_m *Cluster) PodResource(ctx context.Context, podname string) (*types.PodR } // ReallocResource provides a mock function with given fields: ctx, opts -func (_m *Cluster) ReallocResource(ctx context.Context, opts *types.ReallocOptions) (chan *types.ReallocResourceMessage, error) { +func (_m *Cluster) ReallocResource(ctx context.Context, opts *types.ReallocOptions) error { ret := _m.Called(ctx, opts) - var r0 chan *types.ReallocResourceMessage - if rf, ok := ret.Get(0).(func(context.Context, *types.ReallocOptions) chan *types.ReallocResourceMessage); ok { + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, *types.ReallocOptions) error); ok { r0 = rf(ctx, opts) } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(chan *types.ReallocResourceMessage) - } - } - - var r1 error - if rf, ok := ret.Get(1).(func(context.Context, *types.ReallocOptions) error); ok { - r1 = rf(ctx, opts) - } else { - r1 = ret.Error(1) + r0 = ret.Error(0) } - return r0, r1 + return r0 } // RemoveContainer provides a mock function with given fields: ctx, IDs, force, step diff --git a/engine/docker/container.go b/engine/docker/container.go index 21230b565..fce92c982 100644 --- a/engine/docker/container.go +++ b/engine/docker/container.go @@ -113,7 +113,7 @@ func (e *Engine) VirtualizationCreate(ctx context.Context, opts *enginetypes.Vir Tty: opts.Stdin, } - resource := makeResourceSetting(opts.Quota, opts.Memory, opts.CPU, opts.NUMANode, opts.SoftLimit) + resource := makeResourceSetting(opts.Quota, opts.Memory, opts.CPU, opts.NUMANode) // set ulimits resource.Ulimits = []*units.Ulimit{ {Name: "nofile", Soft: 65535, Hard: 65535}, @@ -331,11 +331,11 @@ func (e *Engine) VirtualizationUpdateResource(ctx context.Context, ID string, op } memory := opts.Memory - softLimit := opts.SoftLimit + //softLimit := opts.SoftLimit // unlimited memory if memory == 0 { memory = maxMemory - softLimit = false + //softLimit = false } quota := opts.Quota @@ -357,7 +357,7 @@ func (e *Engine) VirtualizationUpdateResource(ctx context.Context, ID string, op } } - newResource := makeResourceSetting(quota, memory, cpuMap, numaNode, softLimit) + newResource := makeResourceSetting(quota, memory, cpuMap, numaNode) updateConfig := dockercontainer.UpdateConfig{Resources: newResource} _, err := e.client.ContainerUpdate(ctx, ID, updateConfig) return err diff --git a/engine/docker/helper.go b/engine/docker/helper.go index 58cc69460..34bda8c32 100644 --- a/engine/docker/helper.go +++ b/engine/docker/helper.go @@ -99,7 +99,7 @@ func makeMountPaths(opts *enginetypes.VirtualizationCreateOptions) ([]string, ma return binds, volumes } -func makeResourceSetting(cpu float64, memory int64, cpuMap map[string]int64, numaNode string, softlimit bool) dockercontainer.Resources { +func makeResourceSetting(cpu float64, memory int64, cpuMap map[string]int64, numaNode string) dockercontainer.Resources { resource := dockercontainer.Resources{} resource.CPUQuota = 0 @@ -119,16 +119,16 @@ func makeResourceSetting(cpu float64, memory int64, cpuMap map[string]int64, num // numaNode will empty or numaNode resource.CpusetMems = numaNode } - if softlimit { - resource.MemoryReservation = memory - } else { - resource.Memory = memory - resource.MemorySwap = memory - resource.MemoryReservation = memory / 2 - if memory != 0 && memory/2 < int64(units.MiB*4) { - resource.MemoryReservation = int64(units.MiB * 4) - } + //if softlimit { + // resource.MemoryReservation = memory + //} else { + resource.Memory = memory + resource.MemorySwap = memory + resource.MemoryReservation = memory / 2 + if memory != 0 && memory/2 < int64(units.MiB*4) { + resource.MemoryReservation = int64(units.MiB * 4) } + //} return resource } diff --git a/engine/systemd/unit.go b/engine/systemd/unit.go index 568f8463c..63021905f 100644 --- a/engine/systemd/unit.go +++ b/engine/systemd/unit.go @@ -141,17 +141,17 @@ func (b *unitBuilder) buildMemoryLimit() *unitBuilder { return b } - if b.opts.SoftLimit { - b.serviceBuffer = append(b.serviceBuffer, - fmt.Sprintf("ExecStartPre=/usr/bin/cgset -r memory.soft_limit_in_bytes=%d %s", b.opts.Memory, b.cgroupPath()), - ) - - } else { - b.serviceBuffer = append(b.serviceBuffer, - fmt.Sprintf("ExecStartPre=/usr/bin/cgset -r memory.limit_in_bytes=%d %s", b.opts.Memory, b.cgroupPath()), - fmt.Sprintf("ExecStartPre=/usr/bin/cgset -r memory.soft_limit_in_bytes=%d %s", utils.Max(int(b.opts.Memory/2), units.MiB*4), b.cgroupPath()), - ) - } + // if b.opts.SoftLimit { + // b.serviceBuffer = append(b.serviceBuffer, + // fmt.Sprintf("ExecStartPre=/usr/bin/cgset -r memory.soft_limit_in_bytes=%d %s", b.opts.Memory, b.cgroupPath()), + // ) + // + // } else { + b.serviceBuffer = append(b.serviceBuffer, + fmt.Sprintf("ExecStartPre=/usr/bin/cgset -r memory.limit_in_bytes=%d %s", b.opts.Memory, b.cgroupPath()), + fmt.Sprintf("ExecStartPre=/usr/bin/cgset -r memory.soft_limit_in_bytes=%d %s", utils.Max(int(b.opts.Memory/2), units.MiB*4), b.cgroupPath()), + ) + // } return b } diff --git a/engine/types/virtualization.go b/engine/types/virtualization.go index a40ee4e5c..83087b081 100644 --- a/engine/types/virtualization.go +++ b/engine/types/virtualization.go @@ -6,7 +6,6 @@ type VirtualizationResource struct { Quota float64 // for cpu quota Memory int64 // for memory binding Storage int64 - SoftLimit bool // soft limit or not NUMANode string // numa node Volumes []string VolumePlan map[string]map[string]int64 // literal VolumePlan diff --git a/go.mod b/go.mod index 32e935ad0..3d9e1e4a2 100644 --- a/go.mod +++ b/go.mod @@ -41,6 +41,7 @@ require ( golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc golang.org/x/sys v0.0.0-20200812155832-6a926be9bd1d // indirect google.golang.org/grpc v1.28.0 + google.golang.org/protobuf v1.23.0 gopkg.in/alexcesaro/statsd.v2 v2.0.0 // indirect gotest.tools v2.2.0+incompatible // indirect ) diff --git a/go.sum b/go.sum index 20530b73b..f55ab75ee 100644 --- a/go.sum +++ b/go.sum @@ -10,12 +10,14 @@ cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbf cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78 h1:w+iIsaOQNcT7OZ575w+acHgRric5iCyQh+xv+KJ4HB8= github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78/go.mod h1:LmzpDX56iTiv29bbRTIsUNlaFfuhWRQBWjQdVyAevI8= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/CMGS/statsd v0.0.0-20160223095033-48c421b3c1ab h1:/Nl282MSyyUKtYA9gAUF5mlIIdkLkNOBIbE5n9xOxU4= github.com/CMGS/statsd v0.0.0-20160223095033-48c421b3c1ab/go.mod h1:GJO3SGuPXm9A2hpQVV7/wlPr8oP9xxQluI8y99gQu60= +github.com/Microsoft/go-winio v0.4.14 h1:+hMXMk01us9KgxGb7ftKQt2Xpf5hH/yky+TDA+qxleU= github.com/Microsoft/go-winio v0.4.14/go.mod h1:qXqCSQ3Xa7+6tgxaGTIe4Kpcdsi+P8jBhyzoq1bpyYA= github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5 h1:TngWCqHvy9oXAN6lEVMRuU21PR1EtLVZJmdB18Gu3Rw= github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5/go.mod h1:lmUJ/7eu/Q8D7ML55dXQrVaamCz2vxCfdQBasLZfHKk= @@ -102,6 +104,7 @@ github.com/gogo/protobuf v1.3.1 h1:DqDEcV5aeaTmdFBePNpYsp3FlcVH/2ISVVM9Qf8PSls= github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6 h1:ZgQEtGgCBiWRM39fZuwSd1LwSqqSW0hOdXCYYDX0R3I= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -173,6 +176,7 @@ github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvW github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -325,6 +329,7 @@ golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f h1:J5lckAjkw6qYlOZNj90mLYNTEKDvWeuc1yieZ8qUzUE= golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= @@ -412,6 +417,7 @@ golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20190925020647-22afafe3322a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f h1:kDxGY2VmgABOe55qheT/TFqUMtcTHnomIPS1iv3G4Ms= golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -477,6 +483,7 @@ honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= diff --git a/resources/cpumem/cpumem.go b/resources/cpumem/cpumem.go index d0fa1be86..b8d59c1cb 100644 --- a/resources/cpumem/cpumem.go +++ b/resources/cpumem/cpumem.go @@ -1,8 +1,6 @@ package cpumem import ( - "strconv" - "github.com/pkg/errors" resourcetypes "github.com/projecteru2/core/resources/types" "github.com/projecteru2/core/scheduler" @@ -10,68 +8,66 @@ import ( "github.com/projecteru2/core/utils" ) -// NewResourceRequirement . -func NewResourceRequirement(opts types.RawResourceOptions) (resourcetypes.ResourceRequirement, error) { - a := &cpuMemRequirement{ - CPURequest: opts.CPURequest, - CPULimit: opts.CPULimit, +type cpuMemRequest struct { + CPUQuotaRequest float64 + CPUQuotaLimit float64 + CPUBind bool + + memoryRequest int64 + memoryLimit int64 +} + +// MakeRequest . +func MakeRequest(opts types.ResourceOptions) (resourcetypes.ResourceRequest, error) { + cmr := &cpuMemRequest{ + CPUQuotaRequest: opts.CPUQuotaRequest, + CPUQuotaLimit: opts.CPUQuotaLimit, CPUBind: opts.CPUBind, memoryRequest: opts.MemoryRequest, memoryLimit: opts.MemoryLimit, - memorySoftLimit: opts.MemorySoft, } - return a, a.Validate() -} - -// cpuMemRequirement . -type cpuMemRequirement struct { - CPURequest float64 - CPULimit float64 - CPUBind bool - - memoryRequest int64 - memoryLimit int64 - memorySoftLimit bool + return cmr, cmr.Validate() } // Type . -func (a cpuMemRequirement) Type() types.ResourceType { +func (cm cpuMemRequest) Type() types.ResourceType { t := types.ResourceCPU | types.ResourceMemory - if a.CPUBind { + if cm.CPUBind { t |= types.ResourceCPUBind } return t } // Validate . -func (a *cpuMemRequirement) Validate() error { - if a.memoryLimit < 0 || a.memoryRequest < 0 { +func (cm *cpuMemRequest) Validate() error { + if cm.CPUQuotaRequest == 0 && cm.CPUQuotaLimit > 0 { + cm.CPUQuotaRequest = cm.CPUQuotaLimit + } + if cm.memoryLimit < 0 || cm.memoryRequest < 0 { return errors.Wrap(types.ErrBadMemory, "limit or request less than 0") } - if a.memoryRequest == 0 && a.memoryLimit > 0 { - a.memoryRequest = a.memoryLimit + if cm.CPUQuotaLimit < 0 || cm.CPUQuotaRequest < 0 { + return errors.Wrap(types.ErrBadCPU, "limit or request less than 0") } - if a.memoryLimit > 0 && a.memoryRequest > 0 && a.memoryRequest > a.memoryLimit { - return errors.Wrap(types.ErrBadMemory, "limit less than request") + if cm.CPUQuotaRequest == 0 && cm.CPUBind { + return errors.Wrap(types.ErrBadCPU, "unlimited request with bind") } - if a.CPULimit < 0 || a.CPURequest < 0 { - return errors.Wrap(types.ErrBadCPU, "limit or request less than 0") - } - if a.CPURequest == 0 && a.CPULimit > 0 { - a.CPURequest = a.CPULimit + if cm.memoryRequest == 0 && cm.memoryLimit > 0 { + cm.memoryRequest = cm.memoryLimit } - if a.CPURequest > 0 && a.CPULimit > 0 && a.CPURequest > a.CPULimit { - return errors.Wrap(types.ErrBadCPU, "limit less than request") + // 如果需求量大于限制量,悄咪咪的把限制量抬到需求量的水平,做成名义上的软限制 + if cm.memoryLimit > 0 && cm.memoryRequest > 0 && cm.memoryRequest > cm.memoryLimit { + cm.memoryLimit = cm.memoryRequest } - if a.CPURequest == 0 && a.CPUBind { - return errors.Wrap(types.ErrBadCPU, "unlimited request with bind") + if cm.CPUQuotaRequest > 0 && cm.CPUQuotaLimit > 0 && cm.CPUQuotaRequest > cm.CPUQuotaLimit { + cm.CPUQuotaLimit = cm.CPUQuotaRequest } return nil } // MakeScheduler . -func (a cpuMemRequirement) MakeScheduler() resourcetypes.SchedulerV2 { +func (cm cpuMemRequest) MakeScheduler() resourcetypes.SchedulerV2 { return func(nodesInfo []types.NodeInfo) (plans resourcetypes.ResourcePlans, total int, err error) { schedulerV1, err := scheduler.GetSchedulerV1() if err != nil { @@ -79,114 +75,95 @@ func (a cpuMemRequirement) MakeScheduler() resourcetypes.SchedulerV2 { } var CPUPlans map[string][]types.CPUMap - if !a.CPUBind || a.CPURequest == 0 { - nodesInfo, total, err = schedulerV1.SelectMemoryNodes(nodesInfo, a.CPURequest, a.memoryRequest) + if !cm.CPUBind || cm.CPUQuotaRequest == 0 { + nodesInfo, total, err = schedulerV1.SelectMemoryNodes(nodesInfo, cm.CPUQuotaRequest, cm.memoryRequest) } else { - nodesInfo, CPUPlans, total, err = schedulerV1.SelectCPUNodes(nodesInfo, a.CPURequest, a.memoryRequest) + nodesInfo, CPUPlans, total, err = schedulerV1.SelectCPUNodes(nodesInfo, cm.CPUQuotaRequest, cm.memoryRequest) } return ResourcePlans{ - memoryRequest: a.memoryRequest, - memoryLimit: a.memoryLimit, - memorySoftLimit: a.memorySoftLimit, - CPURequest: a.CPURequest, - CPULimit: a.CPULimit, + memoryRequest: cm.memoryRequest, + memoryLimit: cm.memoryLimit, + CPUQuotaRequest: cm.CPUQuotaRequest, + CPUQuotaLimit: cm.CPUQuotaLimit, CPUPlans: CPUPlans, - CPUBind: a.CPUBind, capacity: utils.GetCapacity(nodesInfo), }, total, err } } -// Rate . -func (a cpuMemRequirement) Rate(node types.Node) float64 { - if a.CPUBind { - return a.CPURequest / float64(len(node.InitCPU)) +// Rate for global strategy +func (cm cpuMemRequest) Rate(node types.Node) float64 { + if cm.CPUBind { + return cm.CPUQuotaRequest / float64(len(node.InitCPU)) } - return float64(a.memoryRequest) / float64(node.InitMemCap) + return float64(cm.memoryRequest) / float64(node.InitMemCap) } // ResourcePlans . type ResourcePlans struct { - memoryRequest int64 - memoryLimit int64 - memorySoftLimit bool + memoryRequest int64 + memoryLimit int64 - CPURequest float64 - CPULimit float64 - CPUPlans map[string][]types.CPUMap - CPUBind bool + CPUQuotaRequest float64 + CPUQuotaLimit float64 + CPUPlans map[string][]types.CPUMap capacity map[string]int } // Type . -func (p ResourcePlans) Type() (resourceType types.ResourceType) { +func (rp ResourcePlans) Type() (resourceType types.ResourceType) { resourceType = types.ResourceCPU | types.ResourceMemory - if p.CPUPlans != nil { + if rp.CPUPlans != nil { resourceType |= types.ResourceCPUBind } return resourceType } // Capacity . -func (p ResourcePlans) Capacity() map[string]int { - return p.capacity +func (rp ResourcePlans) Capacity() map[string]int { + return rp.capacity } // ApplyChangesOnNode . -func (p ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { - if p.CPUPlans != nil { +func (rp ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { + if rp.CPUPlans != nil { for _, idx := range indices { - node.CPU.Sub(p.CPUPlans[node.Name][idx]) + node.CPU.Sub(rp.CPUPlans[node.Name][idx]) } } - node.MemCap -= p.memoryRequest * int64(len(indices)) - node.SetCPUUsed(p.CPURequest*float64(len(indices)), types.IncrUsage) + node.MemCap -= rp.memoryRequest * int64(len(indices)) + node.SetCPUUsed(rp.CPUQuotaRequest*float64(len(indices)), types.IncrUsage) } // RollbackChangesOnNode . -func (p ResourcePlans) RollbackChangesOnNode(node *types.Node, indices ...int) { - if p.CPUPlans != nil { +func (rp ResourcePlans) RollbackChangesOnNode(node *types.Node, indices ...int) { + if rp.CPUPlans != nil { for _, idx := range indices { - node.CPU.Add(p.CPUPlans[node.Name][idx]) + node.CPU.Add(rp.CPUPlans[node.Name][idx]) } } - node.MemCap += p.memoryRequest * int64(len(indices)) - node.SetCPUUsed(p.CPURequest*float64(len(indices)), types.DecrUsage) + node.MemCap += rp.memoryRequest * int64(len(indices)) + node.SetCPUUsed(rp.CPUQuotaRequest*float64(len(indices)), types.DecrUsage) } // Dispense . -func (p ResourcePlans) Dispense(opts resourcetypes.DispenseOptions, rsc *types.Resources) error { - rsc.CPUQuotaLimit = p.CPULimit - rsc.CPUQuotaRequest = p.CPURequest - rsc.CPUBind = p.CPUBind - - rsc.MemoryLimit = p.memoryLimit - rsc.MemoryRequest = p.memoryRequest - rsc.MemorySoftLimit = p.memorySoftLimit - - if len(p.CPUPlans) > 0 { - if _, ok := p.CPUPlans[opts.Node.Name]; !ok { - return errors.WithStack(types.ErrInsufficientCPU) +func (rp ResourcePlans) Dispense(opts resourcetypes.DispenseOptions, r *types.ResourceMeta) (*types.ResourceMeta, error) { + r.CPUQuotaLimit = rp.CPUQuotaLimit + r.CPUQuotaRequest = rp.CPUQuotaRequest + r.MemoryLimit = rp.memoryLimit + r.MemoryRequest = rp.memoryRequest + + if len(rp.CPUPlans) > 0 { + if _, ok := rp.CPUPlans[opts.Node.Name]; !ok { + return nil, errors.WithStack(types.ErrInsufficientCPU) } - if len(p.CPUPlans[opts.Node.Name]) <= opts.Index { - return errors.WithStack(types.ErrInsufficientCPU) + if len(rp.CPUPlans[opts.Node.Name]) <= opts.Index { + return nil, errors.WithStack(types.ErrInsufficientCPU) } - rsc.CPURequest = p.CPUPlans[opts.Node.Name][opts.Index] - rsc.NUMANode = opts.Node.GetNUMANode(rsc.CPURequest) - } - - if p.CPULimit > 0 { - rsc.CPULimit = rsc.CPURequest + r.CPU = rp.CPUPlans[opts.Node.Name][opts.Index] + r.NUMANode = opts.Node.GetNUMANode(r.CPU) } - - // special handle when converting from cpu-binding to cpu-unbinding - if len(opts.ExistingInstances) > opts.Index && len(opts.ExistingInstances[opts.Index].CPURequest) > 0 && len(p.CPUPlans) == 0 { - rsc.CPULimit = types.CPUMap{} - for i := 0; i < len(opts.Node.InitCPU); i++ { - rsc.CPULimit[strconv.Itoa(i)] = 0 - } - } - return nil + return r, nil } diff --git a/resources/resources.go b/resources/resources.go index 6bc348027..580a7583c 100644 --- a/resources/resources.go +++ b/resources/resources.go @@ -8,16 +8,16 @@ import ( "github.com/projecteru2/core/types" ) -var registeredFactories = []func(types.RawResourceOptions) (resourcetypes.ResourceRequirement, error){ - cpumem.NewResourceRequirement, - volume.NewResourceRequirement, - storage.NewResourceRequirement, +var registeredFactories = []func(types.ResourceOptions) (resourcetypes.ResourceRequest, error){ + cpumem.MakeRequest, + storage.MakeRequest, + volume.MakeRequest, } -// NewResourceRequirements . -func NewResourceRequirements(opts types.RawResourceOptions) (rrs resourcetypes.ResourceRequirements, err error) { +// MakeRequests . +func MakeRequests(opts types.ResourceOptions) (resourceRequests resourcetypes.ResourceRequests, err error) { for idx, factory := range registeredFactories { - if rrs[idx], err = factory(opts); err != nil { + if resourceRequests[idx], err = factory(opts); err != nil { return } } diff --git a/resources/scheduler.go b/resources/scheduler.go index bececbde5..07821e05f 100644 --- a/resources/scheduler.go +++ b/resources/scheduler.go @@ -9,38 +9,16 @@ import ( log "github.com/sirupsen/logrus" ) -// SelectNodes . -func SelectNodes(rrs resourcetypes.ResourceRequirements, nodeMap map[string]*types.Node) (planMap map[types.ResourceType]resourcetypes.ResourcePlans, total int, scheduleTypes types.ResourceType, err error) { +// SelectNodesByResourceRequests select nodes by resource requests +func SelectNodesByResourceRequests(resourceRequests resourcetypes.ResourceRequests, nodeMap map[string]*types.Node) ( + scheduleType types.ResourceType, + total int, + plans []resourcetypes.ResourcePlans, + err error, +) { total = math.MaxInt16 - subTotal := 0 - planMap = make(map[types.ResourceType]resourcetypes.ResourcePlans) - nodesInfo := getNodesInfo(nodeMap) - log.Debugf("[SelectNode] nodesInfo: %+v", nodesInfo) - for _, rr := range rrs { - scheduler := rr.MakeScheduler() - if planMap[rr.Type()], subTotal, err = scheduler(nodesInfo); err != nil { - return - } - total = utils.Min(total, subTotal) - - // calculate schedule type - if rr.Type()&types.ResourceCPUBind != 0 { - scheduleTypes |= types.ResourceCPU - } - if rr.Type()&types.ResourceScheduledVolume != 0 { - scheduleTypes |= types.ResourceVolume - } - } - - if scheduleTypes == 0 { - scheduleTypes = types.ResourceMemory - } - return -} - -func getNodesInfo(nodes map[string]*types.Node) []types.NodeInfo { - result := []types.NodeInfo{} - for _, node := range nodes { + nodesInfo := []types.NodeInfo{} + for _, node := range nodeMap { nodeInfo := types.NodeInfo{ Name: node.Name, CPUMap: node.CPU, @@ -50,7 +28,28 @@ func getNodesInfo(nodes map[string]*types.Node) []types.NodeInfo { StorageCap: node.StorageCap, Capacity: 0, } - result = append(result, nodeInfo) + nodesInfo = append(nodesInfo, nodeInfo) + } + log.Debugf("[SelectNode] nodesInfo: %+v", nodesInfo) + for _, resourceRequest := range resourceRequests { + plan, subTotal, err := resourceRequest.MakeScheduler()(nodesInfo) + if err != nil { + return scheduleType, total, plans, err + } + plans = append(plans, plan) + total = utils.Min(total, subTotal) + + // calculate schedule type + if resourceRequest.Type()&types.ResourceCPUBind != 0 { + scheduleType |= types.ResourceCPU + } + if resourceRequest.Type()&types.ResourceScheduledVolume != 0 { + scheduleType |= types.ResourceVolume + } + } + + if scheduleType == 0 { + scheduleType = types.ResourceMemory } - return result + return // nolint:nakedret } diff --git a/resources/storage/storage.go b/resources/storage/storage.go index 707401be6..7558a8c2f 100644 --- a/resources/storage/storage.go +++ b/resources/storage/storage.go @@ -8,59 +8,58 @@ import ( "github.com/projecteru2/core/utils" ) -// storageResourceRequirement . -type storageResourceRequirement struct { +type storageRequest struct { request int64 limit int64 } -// NewResourceRequirement . -func NewResourceRequirement(opts types.RawResourceOptions) (resourcetypes.ResourceRequirement, error) { - a := &storageResourceRequirement{ +// MakeRequest . +func MakeRequest(opts types.ResourceOptions) (resourcetypes.ResourceRequest, error) { + sr := &storageRequest{ request: opts.StorageRequest, limit: opts.StorageLimit, } - return a, a.Validate() + return sr, sr.Validate() } // Type . -func (a storageResourceRequirement) Type() types.ResourceType { +func (s storageRequest) Type() types.ResourceType { return types.ResourceStorage } // Validate . -func (a *storageResourceRequirement) Validate() error { - if a.limit > 0 && a.request == 0 { - a.request = a.limit - } - if a.limit < 0 || a.request < 0 { +func (s *storageRequest) Validate() error { + if s.limit < 0 || s.request < 0 { return errors.Wrap(types.ErrBadStorage, "storage limit or request less than 0") } - if a.limit > 0 && a.request > 0 && a.request > a.limit { - return errors.Wrap(types.ErrBadStorage, "storage limit less than request") + if s.limit > 0 && s.request == 0 { + s.request = s.limit + } + if s.limit > 0 && s.request > 0 && s.request > s.limit { + s.limit = s.request // softlimit storage size } return nil } // MakeScheduler . -func (a storageResourceRequirement) MakeScheduler() resourcetypes.SchedulerV2 { +func (s storageRequest) MakeScheduler() resourcetypes.SchedulerV2 { return func(nodesInfo []types.NodeInfo) (plans resourcetypes.ResourcePlans, total int, err error) { schedulerV1, err := scheduler.GetSchedulerV1() if err != nil { return } - nodesInfo, total, err = schedulerV1.SelectStorageNodes(nodesInfo, a.request) + nodesInfo, total, err = schedulerV1.SelectStorageNodes(nodesInfo, s.request) return ResourcePlans{ - request: a.request, - limit: a.limit, + request: s.request, + limit: s.limit, capacity: utils.GetCapacity(nodesInfo), }, total, err } } // Rate . -func (a storageResourceRequirement) Rate(node types.Node) float64 { +func (s storageRequest) Rate(node types.Node) float64 { return float64(0) / float64(node.Volume.Total()) } @@ -72,28 +71,28 @@ type ResourcePlans struct { } // Type . -func (p ResourcePlans) Type() types.ResourceType { +func (rp ResourcePlans) Type() types.ResourceType { return types.ResourceStorage } // Capacity . -func (p ResourcePlans) Capacity() map[string]int { - return p.capacity +func (rp ResourcePlans) Capacity() map[string]int { + return rp.capacity } // ApplyChangesOnNode . -func (p ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { - node.StorageCap -= int64(len(indices)) * p.request +func (rp ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { + node.StorageCap -= int64(len(indices)) * rp.request } // RollbackChangesOnNode . -func (p ResourcePlans) RollbackChangesOnNode(node *types.Node, indices ...int) { - node.StorageCap += int64(len(indices)) * p.request +func (rp ResourcePlans) RollbackChangesOnNode(node *types.Node, indices ...int) { + node.StorageCap += int64(len(indices)) * rp.request } // Dispense . -func (p ResourcePlans) Dispense(opts resourcetypes.DispenseOptions, rsc *types.Resources) error { - rsc.StorageLimit = p.limit - rsc.StorageRequest = p.request - return nil +func (rp ResourcePlans) Dispense(opts resourcetypes.DispenseOptions, r *types.ResourceMeta) (*types.ResourceMeta, error) { + r.StorageLimit = rp.limit + r.StorageRequest = rp.request + return r, nil } diff --git a/resources/types/mocks/ResourcePlans.go b/resources/types/mocks/ResourcePlans.go index 0a55a081b..bf5633171 100644 --- a/resources/types/mocks/ResourcePlans.go +++ b/resources/types/mocks/ResourcePlans.go @@ -1,13 +1,10 @@ -// Code generated by mockery v2.0.0-alpha.2. DO NOT EDIT. +// Code generated by mockery v1.0.0. DO NOT EDIT. package mocks -import ( - resourcestypes "github.com/projecteru2/core/resources/types" - mock "github.com/stretchr/testify/mock" - - types "github.com/projecteru2/core/types" -) +import mock "github.com/stretchr/testify/mock" +import resourcestypes "github.com/projecteru2/core/resources/types" +import types "github.com/projecteru2/core/types" // ResourcePlans is an autogenerated mock type for the ResourcePlans type type ResourcePlans struct { @@ -43,17 +40,26 @@ func (_m *ResourcePlans) Capacity() map[string]int { } // Dispense provides a mock function with given fields: _a0, _a1 -func (_m *ResourcePlans) Dispense(_a0 resourcestypes.DispenseOptions, _a1 *types.Resources) error { +func (_m *ResourcePlans) Dispense(_a0 resourcestypes.DispenseOptions, _a1 *types.ResourceMeta) (*types.ResourceMeta, error) { ret := _m.Called(_a0, _a1) - var r0 error - if rf, ok := ret.Get(0).(func(resourcestypes.DispenseOptions, *types.Resources) error); ok { + var r0 *types.ResourceMeta + if rf, ok := ret.Get(0).(func(resourcestypes.DispenseOptions, *types.ResourceMeta) *types.ResourceMeta); ok { r0 = rf(_a0, _a1) } else { - r0 = ret.Error(0) + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.ResourceMeta) + } } - return r0 + var r1 error + if rf, ok := ret.Get(1).(func(resourcestypes.DispenseOptions, *types.ResourceMeta) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 } // RollbackChangesOnNode provides a mock function with given fields: _a0, _a1 diff --git a/resources/types/types.go b/resources/types/types.go index a15d6c539..0827ea8d3 100644 --- a/resources/types/types.go +++ b/resources/types/types.go @@ -6,11 +6,11 @@ import ( const supported = 3 -// ResourceRequirements . -type ResourceRequirements [supported]ResourceRequirement +// ResourceRequests . +type ResourceRequests [supported]ResourceRequest // ResourceRequirement . -type ResourceRequirement interface { +type ResourceRequest interface { Type() types.ResourceType Validate() error MakeScheduler() SchedulerV2 @@ -23,9 +23,8 @@ type SchedulerV2 func([]types.NodeInfo) (ResourcePlans, int, error) // DispenseOptions . type DispenseOptions struct { *types.Node - ExistingInstances []*types.Container - Index int - HardVolumeBindings types.VolumeBindings + Index int + ExistingInstance *types.Container } // ResourcePlans . @@ -34,5 +33,5 @@ type ResourcePlans interface { Capacity() map[string]int ApplyChangesOnNode(*types.Node, ...int) RollbackChangesOnNode(*types.Node, ...int) - Dispense(DispenseOptions, *types.Resources) error + Dispense(DispenseOptions, *types.ResourceMeta) (*types.ResourceMeta, error) } diff --git a/resources/volume/volume.go b/resources/volume/volume.go index aa98b2d2c..ba7790242 100644 --- a/resources/volume/volume.go +++ b/resources/volume/volume.go @@ -10,38 +10,41 @@ import ( "github.com/projecteru2/core/utils" ) -// volumeResourceApply . -type volumeResourceApply struct { - request [32]types.VolumeBinding - limit [32]types.VolumeBinding - lenReq int - lenLim int +const maxVolumes = 32 + +type volumeRequest struct { + request [maxVolumes]types.VolumeBinding + limit [maxVolumes]types.VolumeBinding + requests int + limits int } -// NewResourceRequirement . -func NewResourceRequirement(opts types.RawResourceOptions) (resourcetypes.ResourceRequirement, error) { - a := &volumeResourceApply{} +// MakeRequest . +func MakeRequest(opts types.ResourceOptions) (resourcetypes.ResourceRequest, error) { + v := &volumeRequest{} sort.Slice(opts.VolumeRequest, func(i, j int) bool { return opts.VolumeRequest[i].ToString(false) < opts.VolumeRequest[j].ToString(false) }) for i, vb := range opts.VolumeRequest { - a.request[i] = *vb + v.request[i] = *vb } - a.lenReq = len(opts.VolumeRequest) - a.lenLim = len(opts.VolumeLimit) + v.requests = len(opts.VolumeRequest) + v.limits = len(opts.VolumeLimit) - sort.Slice(opts.VolumeLimit, func(i, j int) bool { return opts.VolumeLimit[i].ToString(false) < opts.VolumeLimit[j].ToString(false) }) + sort.Slice(opts.VolumeLimit, func(i, j int) bool { + return opts.VolumeLimit[i].ToString(false) < opts.VolumeLimit[j].ToString(false) + }) for i, vb := range opts.VolumeLimit { - a.limit[i] = *vb + v.limit[i] = *vb } - return a, a.Validate() + return v, v.Validate() } // Type . -func (a volumeResourceApply) Type() types.ResourceType { +func (v volumeRequest) Type() types.ResourceType { t := types.ResourceVolume - for i := 0; i < a.lenReq; i++ { - if a.request[i].RequireSchedule() { + for i := 0; i < v.requests; i++ { + if v.request[i].RequireSchedule() { t |= types.ResourceScheduledVolume break } @@ -50,82 +53,82 @@ func (a volumeResourceApply) Type() types.ResourceType { } // Validate . -func (a *volumeResourceApply) Validate() error { - if a.lenReq == 0 && a.lenLim > 0 { - a.request = a.limit - a.lenReq = a.lenLim +func (v *volumeRequest) Validate() error { + if v.requests == 0 && v.limits > 0 { + v.request = v.limit + v.requests = v.limits } - if a.lenReq != a.lenLim { + if v.requests != v.limits { return errors.Wrap(types.ErrBadVolume, "different length of request and limit") } - for i := 0; i < a.lenReq; i++ { - req, lim := a.request[i], a.limit[i] + for i := 0; i < v.requests; i++ { + req, lim := v.request[i], v.limit[i] if req.Source != lim.Source || req.Destination != lim.Destination || req.Flags != lim.Flags { return errors.Wrap(types.ErrBadVolume, "request and limit not match") } if req.SizeInBytes > 0 && lim.SizeInBytes > 0 && req.SizeInBytes > lim.SizeInBytes { - return errors.Wrap(types.ErrBadVolume, "request size less than limit size ") + v.limit[i].SizeInBytes = req.SizeInBytes } } return nil } // MakeScheduler . -func (a volumeResourceApply) MakeScheduler() resourcetypes.SchedulerV2 { +func (v volumeRequest) MakeScheduler() resourcetypes.SchedulerV2 { return func(nodesInfo []types.NodeInfo) (plans resourcetypes.ResourcePlans, total int, err error) { schedulerV1, err := scheduler.GetSchedulerV1() if err != nil { return } - req, lim := types.VolumeBindings{}, types.VolumeBindings{} - for i := 0; i < a.lenReq; i++ { - req = append(req, &a.request[i]) - lim = append(lim, &a.limit[i]) + request, limit := types.VolumeBindings{}, types.VolumeBindings{} + for i := 0; i < v.requests; i++ { + request = append(request, &v.request[i]) + limit = append(limit, &v.limit[i]) } - nodesInfo, volumePlans, total, err := schedulerV1.SelectVolumeNodes(nodesInfo, req) + nodesInfo, volumePlans, total, err := schedulerV1.SelectVolumeNodes(nodesInfo, request) return ResourcePlans{ capacity: utils.GetCapacity(nodesInfo), - req: req, - lim: lim, - PlanReq: volumePlans, + request: request, + limit: limit, + plan: volumePlans, }, total, err } } // Rate . -func (a volumeResourceApply) Rate(node types.Node) float64 { +func (v volumeRequest) Rate(node types.Node) float64 { return float64(node.VolumeUsed) / float64(node.Volume.Total()) } // ResourcePlans . type ResourcePlans struct { capacity map[string]int - req types.VolumeBindings - lim types.VolumeBindings - PlanReq map[string][]types.VolumePlan + request types.VolumeBindings + limit types.VolumeBindings + plan map[string][]types.VolumePlan } // Type . -func (p ResourcePlans) Type() types.ResourceType { +func (rp ResourcePlans) Type() types.ResourceType { return types.ResourceVolume } // Capacity . -func (p ResourcePlans) Capacity() map[string]int { - return p.capacity +func (rp ResourcePlans) Capacity() map[string]int { + return rp.capacity } // ApplyChangesOnNode . -func (p ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { - if len(p.PlanReq) == 0 { +func (rp ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { + if len(rp.plan) == 0 { return } volumeCost := types.VolumeMap{} for _, idx := range indices { - plans, ok := p.PlanReq[node.Name] + plans, ok := rp.plan[node.Name] if !ok { continue } @@ -136,76 +139,66 @@ func (p ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { } // RollbackChangesOnNode . -func (p ResourcePlans) RollbackChangesOnNode(node *types.Node, indices ...int) { - if len(p.PlanReq) == 0 { +func (rp ResourcePlans) RollbackChangesOnNode(node *types.Node, indices ...int) { + if len(rp.plan) == 0 { return } volumeCost := types.VolumeMap{} for _, idx := range indices { - volumeCost.Add(p.PlanReq[node.Name][idx].IntoVolumeMap()) + volumeCost.Add(rp.plan[node.Name][idx].IntoVolumeMap()) } node.Volume.Add(volumeCost) node.SetVolumeUsed(volumeCost.Total(), types.DecrUsage) } // Dispense . -func (p ResourcePlans) Dispense(opts resourcetypes.DispenseOptions, rsc *types.Resources) error { - if len(p.PlanReq) == 0 { - return nil +func (rp ResourcePlans) Dispense(opts resourcetypes.DispenseOptions, r *types.ResourceMeta) (*types.ResourceMeta, error) { + if len(rp.plan) == 0 { + return r, nil } - rsc.VolumeRequest = p.req - rsc.VolumePlanRequest = p.PlanReq[opts.Node.Name][opts.Index] + r.VolumeRequest = rp.request + r.VolumePlanRequest = rp.plan[opts.Node.Name][opts.Index] // if there are existing ones, ensure new volumes are compatible - if len(opts.ExistingInstances) > 0 { - plans := map[*types.Container]types.VolumePlan{} - Searching: - for _, plan := range p.PlanReq[opts.Node.Name] { - for _, container := range opts.ExistingInstances { - if _, ok := plans[container]; !ok && plan.Compatible(container.VolumePlanRequest) { - plans[container] = plan - if len(plans) == len(opts.ExistingInstances) { - break Searching - } - break - } + if opts.ExistingInstance != nil { + found := false + for _, plan := range rp.plan[opts.Node.Name] { + if plan.Compatible(opts.ExistingInstance.VolumePlanRequest) { + r.VolumePlanRequest = plan + found = true + break } } - if len(plans) < len(opts.ExistingInstances) { - return errors.Wrap(types.ErrInsufficientVolume, "incompatible volume plans") + if !found { + return nil, errors.Wrap(types.ErrInsufficientVolume, "incompatible volume plans") } - - rsc.VolumePlanRequest = plans[opts.ExistingInstances[opts.Index]] } // fix plans while limit > request - rsc.VolumeLimit = p.lim - rsc.VolumePlanLimit = types.VolumePlan{} - for i := range p.req { - req, lim := p.req[i], p.lim[i] - if !req.RequireSchedule() { + r.VolumeLimit = rp.limit + r.VolumePlanLimit = types.VolumePlan{} + for i := range rp.request { + request, limit := rp.request[i], rp.limit[i] + if !request.RequireSchedule() { continue } - if lim.SizeInBytes > req.SizeInBytes { - p := rsc.VolumePlanRequest[*req] - rsc.VolumePlanLimit[*lim] = types.VolumeMap{p.GetResourceID(): p.GetRation() + lim.SizeInBytes - req.SizeInBytes} + if limit.SizeInBytes > request.SizeInBytes { + p := r.VolumePlanRequest[*request] + r.VolumePlanLimit[*limit] = types.VolumeMap{p.GetResourceID(): p.GetRation() + limit.SizeInBytes - request.SizeInBytes} } else { - rsc.VolumePlanLimit[*lim] = rsc.VolumePlanRequest[*req] + r.VolumePlanLimit[*limit] = r.VolumePlanRequest[*request] } } - // append hard vbs - if opts.HardVolumeBindings != nil { - rsc.VolumeRequest = append(rsc.VolumeRequest, opts.HardVolumeBindings...) - rsc.VolumeLimit = append(rsc.VolumeLimit, opts.HardVolumeBindings...) - } - // judge if volume changed - if len(opts.ExistingInstances) > 0 && !rsc.VolumeLimit.IsEqual(opts.ExistingInstances[opts.Index].VolumeLimit) { - rsc.VolumeChanged = true - } - return nil + r.VolumeChanged = opts.ExistingInstance != nil && !r.VolumeLimit.IsEqual(opts.ExistingInstance.VolumeLimit) + return r, nil +} + +// GetPlan return volume plans by nodename +func (rp ResourcePlans) GetPlan(nodename string) []types.VolumePlan { + return rp.plan[nodename] } diff --git a/rpc/gen/core.pb.go b/rpc/gen/core.pb.go index 0b8f346d6..31670d10e 100644 --- a/rpc/gen/core.pb.go +++ b/rpc/gen/core.pb.go @@ -1,28 +1,33 @@ // Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.23.0 +// protoc v3.12.4 // source: core.proto package pb import ( context "context" - fmt "fmt" proto "github.com/golang/protobuf/proto" grpc "google.golang.org/grpc" codes "google.golang.org/grpc/codes" status "google.golang.org/grpc/status" - math "math" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" ) -// Reference imports to suppress errors if they are not otherwise used. -var _ = proto.Marshal -var _ = fmt.Errorf -var _ = math.Inf +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) -// This is a compile-time assertion to ensure that this generated file -// is compatible with the proto package it is being compiled against. -// A compilation error at this line likely means your copy of the -// proto package needs to be updated. -const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package +// This is a compile-time assertion that a sufficiently up-to-date version +// of the legacy proto package is being used. +const _ = proto.ProtoPackageIsVersion4 type TriOpt int32 @@ -32,24 +37,45 @@ const ( TriOpt_FALSE TriOpt = 2 ) -var TriOpt_name = map[int32]string{ - 0: "KEEP", - 1: "TRUE", - 2: "FALSE", -} +// Enum value maps for TriOpt. +var ( + TriOpt_name = map[int32]string{ + 0: "KEEP", + 1: "TRUE", + 2: "FALSE", + } + TriOpt_value = map[string]int32{ + "KEEP": 0, + "TRUE": 1, + "FALSE": 2, + } +) -var TriOpt_value = map[string]int32{ - "KEEP": 0, - "TRUE": 1, - "FALSE": 2, +func (x TriOpt) Enum() *TriOpt { + p := new(TriOpt) + *p = x + return p } func (x TriOpt) String() string { - return proto.EnumName(TriOpt_name, int32(x)) + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) } +func (TriOpt) Descriptor() protoreflect.EnumDescriptor { + return file_core_proto_enumTypes[0].Descriptor() +} + +func (TriOpt) Type() protoreflect.EnumType { + return &file_core_proto_enumTypes[0] +} + +func (x TriOpt) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TriOpt.Descriptor instead. func (TriOpt) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{0} + return file_core_proto_rawDescGZIP(), []int{0} } type BuildImageOptions_BuildMethod int32 @@ -60,24 +86,45 @@ const ( BuildImageOptions_EXIST BuildImageOptions_BuildMethod = 2 ) -var BuildImageOptions_BuildMethod_name = map[int32]string{ - 0: "SCM", - 1: "RAW", - 2: "EXIST", -} +// Enum value maps for BuildImageOptions_BuildMethod. +var ( + BuildImageOptions_BuildMethod_name = map[int32]string{ + 0: "SCM", + 1: "RAW", + 2: "EXIST", + } + BuildImageOptions_BuildMethod_value = map[string]int32{ + "SCM": 0, + "RAW": 1, + "EXIST": 2, + } +) -var BuildImageOptions_BuildMethod_value = map[string]int32{ - "SCM": 0, - "RAW": 1, - "EXIST": 2, +func (x BuildImageOptions_BuildMethod) Enum() *BuildImageOptions_BuildMethod { + p := new(BuildImageOptions_BuildMethod) + *p = x + return p } func (x BuildImageOptions_BuildMethod) String() string { - return proto.EnumName(BuildImageOptions_BuildMethod_name, int32(x)) + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (BuildImageOptions_BuildMethod) Descriptor() protoreflect.EnumDescriptor { + return file_core_proto_enumTypes[1].Descriptor() +} + +func (BuildImageOptions_BuildMethod) Type() protoreflect.EnumType { + return &file_core_proto_enumTypes[1] +} + +func (x BuildImageOptions_BuildMethod) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) } +// Deprecated: Use BuildImageOptions_BuildMethod.Descriptor instead. func (BuildImageOptions_BuildMethod) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{39, 0} + return file_core_proto_rawDescGZIP(), []int{39, 0} } type DeployOptions_Strategy int32 @@ -89,5278 +136,7605 @@ const ( DeployOptions_GLOBAL DeployOptions_Strategy = 3 ) -var DeployOptions_Strategy_name = map[int32]string{ - 0: "AUTO", - 1: "FILL", - 2: "EACH", - 3: "GLOBAL", -} +// Enum value maps for DeployOptions_Strategy. +var ( + DeployOptions_Strategy_name = map[int32]string{ + 0: "AUTO", + 1: "FILL", + 2: "EACH", + 3: "GLOBAL", + } + DeployOptions_Strategy_value = map[string]int32{ + "AUTO": 0, + "FILL": 1, + "EACH": 2, + "GLOBAL": 3, + } +) -var DeployOptions_Strategy_value = map[string]int32{ - "AUTO": 0, - "FILL": 1, - "EACH": 2, - "GLOBAL": 3, +func (x DeployOptions_Strategy) Enum() *DeployOptions_Strategy { + p := new(DeployOptions_Strategy) + *p = x + return p } func (x DeployOptions_Strategy) String() string { - return proto.EnumName(DeployOptions_Strategy_name, int32(x)) + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) } -func (DeployOptions_Strategy) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{44, 0} +func (DeployOptions_Strategy) Descriptor() protoreflect.EnumDescriptor { + return file_core_proto_enumTypes[2].Descriptor() } -type Empty struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` +func (DeployOptions_Strategy) Type() protoreflect.EnumType { + return &file_core_proto_enumTypes[2] } -func (m *Empty) Reset() { *m = Empty{} } -func (m *Empty) String() string { return proto.CompactTextString(m) } -func (*Empty) ProtoMessage() {} -func (*Empty) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{0} +func (x DeployOptions_Strategy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) } -func (m *Empty) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Empty.Unmarshal(m, b) -} -func (m *Empty) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Empty.Marshal(b, m, deterministic) +// Deprecated: Use DeployOptions_Strategy.Descriptor instead. +func (DeployOptions_Strategy) EnumDescriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{47, 0} } -func (m *Empty) XXX_Merge(src proto.Message) { - xxx_messageInfo_Empty.Merge(m, src) + +type Empty struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields } -func (m *Empty) XXX_Size() int { - return xxx_messageInfo_Empty.Size(m) + +func (x *Empty) Reset() { + *x = Empty{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Empty) XXX_DiscardUnknown() { - xxx_messageInfo_Empty.DiscardUnknown(m) + +func (x *Empty) String() string { + return protoimpl.X.MessageStringOf(x) } -var xxx_messageInfo_Empty proto.InternalMessageInfo +func (*Empty) ProtoMessage() {} -type CoreInfo struct { - Version string `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` - Revison string `protobuf:"bytes,2,opt,name=revison,proto3" json:"revison,omitempty"` - BuildAt string `protobuf:"bytes,3,opt,name=build_at,json=buildAt,proto3" json:"build_at,omitempty"` - GolangVersion string `protobuf:"bytes,4,opt,name=golang_version,json=golangVersion,proto3" json:"golang_version,omitempty"` - OsArch string `protobuf:"bytes,5,opt,name=os_arch,json=osArch,proto3" json:"os_arch,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CoreInfo) Reset() { *m = CoreInfo{} } -func (m *CoreInfo) String() string { return proto.CompactTextString(m) } -func (*CoreInfo) ProtoMessage() {} -func (*CoreInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{1} +func (x *Empty) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -func (m *CoreInfo) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CoreInfo.Unmarshal(m, b) +// Deprecated: Use Empty.ProtoReflect.Descriptor instead. +func (*Empty) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{0} } -func (m *CoreInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CoreInfo.Marshal(b, m, deterministic) + +type CoreInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Version string `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` + Revison string `protobuf:"bytes,2,opt,name=revison,proto3" json:"revison,omitempty"` + BuildAt string `protobuf:"bytes,3,opt,name=build_at,json=buildAt,proto3" json:"build_at,omitempty"` + GolangVersion string `protobuf:"bytes,4,opt,name=golang_version,json=golangVersion,proto3" json:"golang_version,omitempty"` + OsArch string `protobuf:"bytes,5,opt,name=os_arch,json=osArch,proto3" json:"os_arch,omitempty"` } -func (m *CoreInfo) XXX_Merge(src proto.Message) { - xxx_messageInfo_CoreInfo.Merge(m, src) + +func (x *CoreInfo) Reset() { + *x = CoreInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *CoreInfo) XXX_Size() int { - return xxx_messageInfo_CoreInfo.Size(m) + +func (x *CoreInfo) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *CoreInfo) XXX_DiscardUnknown() { - xxx_messageInfo_CoreInfo.DiscardUnknown(m) + +func (*CoreInfo) ProtoMessage() {} + +func (x *CoreInfo) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_CoreInfo proto.InternalMessageInfo +// Deprecated: Use CoreInfo.ProtoReflect.Descriptor instead. +func (*CoreInfo) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{1} +} -func (m *CoreInfo) GetVersion() string { - if m != nil { - return m.Version +func (x *CoreInfo) GetVersion() string { + if x != nil { + return x.Version } return "" } -func (m *CoreInfo) GetRevison() string { - if m != nil { - return m.Revison +func (x *CoreInfo) GetRevison() string { + if x != nil { + return x.Revison } return "" } -func (m *CoreInfo) GetBuildAt() string { - if m != nil { - return m.BuildAt +func (x *CoreInfo) GetBuildAt() string { + if x != nil { + return x.BuildAt } return "" } -func (m *CoreInfo) GetGolangVersion() string { - if m != nil { - return m.GolangVersion +func (x *CoreInfo) GetGolangVersion() string { + if x != nil { + return x.GolangVersion } return "" } -func (m *CoreInfo) GetOsArch() string { - if m != nil { - return m.OsArch +func (x *CoreInfo) GetOsArch() string { + if x != nil { + return x.OsArch } return "" } type ServiceStatus struct { - Addresses []string `protobuf:"bytes,1,rep,name=addresses,proto3" json:"addresses,omitempty"` - IntervalInSecond int64 `protobuf:"varint,2,opt,name=interval_in_second,json=intervalInSecond,proto3" json:"interval_in_second,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ServiceStatus) Reset() { *m = ServiceStatus{} } -func (m *ServiceStatus) String() string { return proto.CompactTextString(m) } -func (*ServiceStatus) ProtoMessage() {} -func (*ServiceStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{2} + Addresses []string `protobuf:"bytes,1,rep,name=addresses,proto3" json:"addresses,omitempty"` + IntervalInSecond int64 `protobuf:"varint,2,opt,name=interval_in_second,json=intervalInSecond,proto3" json:"interval_in_second,omitempty"` } -func (m *ServiceStatus) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ServiceStatus.Unmarshal(m, b) -} -func (m *ServiceStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ServiceStatus.Marshal(b, m, deterministic) -} -func (m *ServiceStatus) XXX_Merge(src proto.Message) { - xxx_messageInfo_ServiceStatus.Merge(m, src) +func (x *ServiceStatus) Reset() { + *x = ServiceStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ServiceStatus) XXX_Size() int { - return xxx_messageInfo_ServiceStatus.Size(m) + +func (x *ServiceStatus) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ServiceStatus) XXX_DiscardUnknown() { - xxx_messageInfo_ServiceStatus.DiscardUnknown(m) + +func (*ServiceStatus) ProtoMessage() {} + +func (x *ServiceStatus) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ServiceStatus proto.InternalMessageInfo +// Deprecated: Use ServiceStatus.ProtoReflect.Descriptor instead. +func (*ServiceStatus) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{2} +} -func (m *ServiceStatus) GetAddresses() []string { - if m != nil { - return m.Addresses +func (x *ServiceStatus) GetAddresses() []string { + if x != nil { + return x.Addresses } return nil } -func (m *ServiceStatus) GetIntervalInSecond() int64 { - if m != nil { - return m.IntervalInSecond +func (x *ServiceStatus) GetIntervalInSecond() int64 { + if x != nil { + return x.IntervalInSecond } return 0 } type ListContainersOptions struct { - Appname string `protobuf:"bytes,1,opt,name=appname,proto3" json:"appname,omitempty"` - Entrypoint string `protobuf:"bytes,2,opt,name=entrypoint,proto3" json:"entrypoint,omitempty"` - Nodename string `protobuf:"bytes,3,opt,name=nodename,proto3" json:"nodename,omitempty"` - Labels map[string]string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Limit int64 `protobuf:"varint,5,opt,name=limit,proto3" json:"limit,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ListContainersOptions) Reset() { *m = ListContainersOptions{} } -func (m *ListContainersOptions) String() string { return proto.CompactTextString(m) } -func (*ListContainersOptions) ProtoMessage() {} -func (*ListContainersOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{3} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ListContainersOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ListContainersOptions.Unmarshal(m, b) + Appname string `protobuf:"bytes,1,opt,name=appname,proto3" json:"appname,omitempty"` + Entrypoint string `protobuf:"bytes,2,opt,name=entrypoint,proto3" json:"entrypoint,omitempty"` + Nodename string `protobuf:"bytes,3,opt,name=nodename,proto3" json:"nodename,omitempty"` + Labels map[string]string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Limit int64 `protobuf:"varint,5,opt,name=limit,proto3" json:"limit,omitempty"` } -func (m *ListContainersOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ListContainersOptions.Marshal(b, m, deterministic) -} -func (m *ListContainersOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_ListContainersOptions.Merge(m, src) + +func (x *ListContainersOptions) Reset() { + *x = ListContainersOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ListContainersOptions) XXX_Size() int { - return xxx_messageInfo_ListContainersOptions.Size(m) + +func (x *ListContainersOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ListContainersOptions) XXX_DiscardUnknown() { - xxx_messageInfo_ListContainersOptions.DiscardUnknown(m) + +func (*ListContainersOptions) ProtoMessage() {} + +func (x *ListContainersOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ListContainersOptions proto.InternalMessageInfo +// Deprecated: Use ListContainersOptions.ProtoReflect.Descriptor instead. +func (*ListContainersOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{3} +} -func (m *ListContainersOptions) GetAppname() string { - if m != nil { - return m.Appname +func (x *ListContainersOptions) GetAppname() string { + if x != nil { + return x.Appname } return "" } -func (m *ListContainersOptions) GetEntrypoint() string { - if m != nil { - return m.Entrypoint +func (x *ListContainersOptions) GetEntrypoint() string { + if x != nil { + return x.Entrypoint } return "" } -func (m *ListContainersOptions) GetNodename() string { - if m != nil { - return m.Nodename +func (x *ListContainersOptions) GetNodename() string { + if x != nil { + return x.Nodename } return "" } -func (m *ListContainersOptions) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *ListContainersOptions) GetLabels() map[string]string { + if x != nil { + return x.Labels } return nil } -func (m *ListContainersOptions) GetLimit() int64 { - if m != nil { - return m.Limit +func (x *ListContainersOptions) GetLimit() int64 { + if x != nil { + return x.Limit } return 0 } // 对的, protobuf 就是这样... type Pod struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Desc string `protobuf:"bytes,2,opt,name=desc,proto3" json:"desc,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Pod) Reset() { *m = Pod{} } -func (m *Pod) String() string { return proto.CompactTextString(m) } -func (*Pod) ProtoMessage() {} -func (*Pod) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{4} + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Desc string `protobuf:"bytes,2,opt,name=desc,proto3" json:"desc,omitempty"` } -func (m *Pod) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Pod.Unmarshal(m, b) -} -func (m *Pod) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Pod.Marshal(b, m, deterministic) -} -func (m *Pod) XXX_Merge(src proto.Message) { - xxx_messageInfo_Pod.Merge(m, src) +func (x *Pod) Reset() { + *x = Pod{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Pod) XXX_Size() int { - return xxx_messageInfo_Pod.Size(m) + +func (x *Pod) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Pod) XXX_DiscardUnknown() { - xxx_messageInfo_Pod.DiscardUnknown(m) + +func (*Pod) ProtoMessage() {} + +func (x *Pod) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_Pod proto.InternalMessageInfo +// Deprecated: Use Pod.ProtoReflect.Descriptor instead. +func (*Pod) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{4} +} -func (m *Pod) GetName() string { - if m != nil { - return m.Name +func (x *Pod) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *Pod) GetDesc() string { - if m != nil { - return m.Desc +func (x *Pod) GetDesc() string { + if x != nil { + return x.Desc } return "" } type Pods struct { - Pods []*Pod `protobuf:"bytes,1,rep,name=pods,proto3" json:"pods,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Pods) Reset() { *m = Pods{} } -func (m *Pods) String() string { return proto.CompactTextString(m) } -func (*Pods) ProtoMessage() {} -func (*Pods) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{5} + Pods []*Pod `protobuf:"bytes,1,rep,name=pods,proto3" json:"pods,omitempty"` } -func (m *Pods) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Pods.Unmarshal(m, b) -} -func (m *Pods) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Pods.Marshal(b, m, deterministic) -} -func (m *Pods) XXX_Merge(src proto.Message) { - xxx_messageInfo_Pods.Merge(m, src) +func (x *Pods) Reset() { + *x = Pods{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Pods) XXX_Size() int { - return xxx_messageInfo_Pods.Size(m) + +func (x *Pods) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Pods) XXX_DiscardUnknown() { - xxx_messageInfo_Pods.DiscardUnknown(m) + +func (*Pods) ProtoMessage() {} + +func (x *Pods) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_Pods proto.InternalMessageInfo +// Deprecated: Use Pods.ProtoReflect.Descriptor instead. +func (*Pods) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{5} +} -func (m *Pods) GetPods() []*Pod { - if m != nil { - return m.Pods +func (x *Pods) GetPods() []*Pod { + if x != nil { + return x.Pods } return nil } type PodResource struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - CpuPercents map[string]float64 `protobuf:"bytes,2,rep,name=cpu_percents,json=cpuPercents,proto3" json:"cpu_percents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` - MemoryPercents map[string]float64 `protobuf:"bytes,3,rep,name=memory_percents,json=memoryPercents,proto3" json:"memory_percents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` - Verifications map[string]bool `protobuf:"bytes,4,rep,name=verifications,proto3" json:"verifications,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Details map[string]string `protobuf:"bytes,5,rep,name=details,proto3" json:"details,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - StoragePercents map[string]float64 `protobuf:"bytes,6,rep,name=storage_percents,json=storagePercents,proto3" json:"storage_percents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` - VolumePercents map[string]float64 `protobuf:"bytes,7,rep,name=volume_percents,json=volumePercents,proto3" json:"volume_percents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *PodResource) Reset() { *m = PodResource{} } -func (m *PodResource) String() string { return proto.CompactTextString(m) } -func (*PodResource) ProtoMessage() {} -func (*PodResource) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{6} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *PodResource) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_PodResource.Unmarshal(m, b) -} -func (m *PodResource) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_PodResource.Marshal(b, m, deterministic) + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + CpuPercents map[string]float64 `protobuf:"bytes,2,rep,name=cpu_percents,json=cpuPercents,proto3" json:"cpu_percents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` + MemoryPercents map[string]float64 `protobuf:"bytes,3,rep,name=memory_percents,json=memoryPercents,proto3" json:"memory_percents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` + Verifications map[string]bool `protobuf:"bytes,4,rep,name=verifications,proto3" json:"verifications,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Details map[string]string `protobuf:"bytes,5,rep,name=details,proto3" json:"details,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + StoragePercents map[string]float64 `protobuf:"bytes,6,rep,name=storage_percents,json=storagePercents,proto3" json:"storage_percents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` + VolumePercents map[string]float64 `protobuf:"bytes,7,rep,name=volume_percents,json=volumePercents,proto3" json:"volume_percents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` } -func (m *PodResource) XXX_Merge(src proto.Message) { - xxx_messageInfo_PodResource.Merge(m, src) + +func (x *PodResource) Reset() { + *x = PodResource{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *PodResource) XXX_Size() int { - return xxx_messageInfo_PodResource.Size(m) + +func (x *PodResource) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *PodResource) XXX_DiscardUnknown() { - xxx_messageInfo_PodResource.DiscardUnknown(m) + +func (*PodResource) ProtoMessage() {} + +func (x *PodResource) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_PodResource proto.InternalMessageInfo +// Deprecated: Use PodResource.ProtoReflect.Descriptor instead. +func (*PodResource) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{6} +} -func (m *PodResource) GetName() string { - if m != nil { - return m.Name +func (x *PodResource) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *PodResource) GetCpuPercents() map[string]float64 { - if m != nil { - return m.CpuPercents +func (x *PodResource) GetCpuPercents() map[string]float64 { + if x != nil { + return x.CpuPercents } return nil } -func (m *PodResource) GetMemoryPercents() map[string]float64 { - if m != nil { - return m.MemoryPercents +func (x *PodResource) GetMemoryPercents() map[string]float64 { + if x != nil { + return x.MemoryPercents } return nil } -func (m *PodResource) GetVerifications() map[string]bool { - if m != nil { - return m.Verifications +func (x *PodResource) GetVerifications() map[string]bool { + if x != nil { + return x.Verifications } return nil } -func (m *PodResource) GetDetails() map[string]string { - if m != nil { - return m.Details +func (x *PodResource) GetDetails() map[string]string { + if x != nil { + return x.Details } return nil } -func (m *PodResource) GetStoragePercents() map[string]float64 { - if m != nil { - return m.StoragePercents +func (x *PodResource) GetStoragePercents() map[string]float64 { + if x != nil { + return x.StoragePercents } return nil } -func (m *PodResource) GetVolumePercents() map[string]float64 { - if m != nil { - return m.VolumePercents +func (x *PodResource) GetVolumePercents() map[string]float64 { + if x != nil { + return x.VolumePercents } return nil } type NodeResource struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - CpuPercent float64 `protobuf:"fixed64,2,opt,name=cpu_percent,json=cpuPercent,proto3" json:"cpu_percent,omitempty"` - MemoryPercent float64 `protobuf:"fixed64,3,opt,name=memory_percent,json=memoryPercent,proto3" json:"memory_percent,omitempty"` - Verification bool `protobuf:"varint,4,opt,name=verification,proto3" json:"verification,omitempty"` - Details []string `protobuf:"bytes,5,rep,name=details,proto3" json:"details,omitempty"` - StoragePercent float64 `protobuf:"fixed64,6,opt,name=storage_percent,json=storagePercent,proto3" json:"storage_percent,omitempty"` - VolumePercent float64 `protobuf:"fixed64,7,opt,name=volume_percent,json=volumePercent,proto3" json:"volume_percent,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *NodeResource) Reset() { *m = NodeResource{} } -func (m *NodeResource) String() string { return proto.CompactTextString(m) } -func (*NodeResource) ProtoMessage() {} -func (*NodeResource) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{7} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *NodeResource) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_NodeResource.Unmarshal(m, b) + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + CpuPercent float64 `protobuf:"fixed64,2,opt,name=cpu_percent,json=cpuPercent,proto3" json:"cpu_percent,omitempty"` + MemoryPercent float64 `protobuf:"fixed64,3,opt,name=memory_percent,json=memoryPercent,proto3" json:"memory_percent,omitempty"` + Verification bool `protobuf:"varint,4,opt,name=verification,proto3" json:"verification,omitempty"` + Details []string `protobuf:"bytes,5,rep,name=details,proto3" json:"details,omitempty"` + StoragePercent float64 `protobuf:"fixed64,6,opt,name=storage_percent,json=storagePercent,proto3" json:"storage_percent,omitempty"` + VolumePercent float64 `protobuf:"fixed64,7,opt,name=volume_percent,json=volumePercent,proto3" json:"volume_percent,omitempty"` } -func (m *NodeResource) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_NodeResource.Marshal(b, m, deterministic) -} -func (m *NodeResource) XXX_Merge(src proto.Message) { - xxx_messageInfo_NodeResource.Merge(m, src) + +func (x *NodeResource) Reset() { + *x = NodeResource{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *NodeResource) XXX_Size() int { - return xxx_messageInfo_NodeResource.Size(m) + +func (x *NodeResource) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *NodeResource) XXX_DiscardUnknown() { - xxx_messageInfo_NodeResource.DiscardUnknown(m) + +func (*NodeResource) ProtoMessage() {} + +func (x *NodeResource) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_NodeResource proto.InternalMessageInfo +// Deprecated: Use NodeResource.ProtoReflect.Descriptor instead. +func (*NodeResource) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{7} +} -func (m *NodeResource) GetName() string { - if m != nil { - return m.Name +func (x *NodeResource) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *NodeResource) GetCpuPercent() float64 { - if m != nil { - return m.CpuPercent +func (x *NodeResource) GetCpuPercent() float64 { + if x != nil { + return x.CpuPercent } return 0 } -func (m *NodeResource) GetMemoryPercent() float64 { - if m != nil { - return m.MemoryPercent +func (x *NodeResource) GetMemoryPercent() float64 { + if x != nil { + return x.MemoryPercent } return 0 } -func (m *NodeResource) GetVerification() bool { - if m != nil { - return m.Verification +func (x *NodeResource) GetVerification() bool { + if x != nil { + return x.Verification } return false } -func (m *NodeResource) GetDetails() []string { - if m != nil { - return m.Details +func (x *NodeResource) GetDetails() []string { + if x != nil { + return x.Details } return nil } -func (m *NodeResource) GetStoragePercent() float64 { - if m != nil { - return m.StoragePercent +func (x *NodeResource) GetStoragePercent() float64 { + if x != nil { + return x.StoragePercent } return 0 } -func (m *NodeResource) GetVolumePercent() float64 { - if m != nil { - return m.VolumePercent +func (x *NodeResource) GetVolumePercent() float64 { + if x != nil { + return x.VolumePercent } return 0 } type ListNetworkOptions struct { - Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` - Driver string `protobuf:"bytes,2,opt,name=driver,proto3" json:"driver,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ListNetworkOptions) Reset() { *m = ListNetworkOptions{} } -func (m *ListNetworkOptions) String() string { return proto.CompactTextString(m) } -func (*ListNetworkOptions) ProtoMessage() {} -func (*ListNetworkOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{8} + Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` + Driver string `protobuf:"bytes,2,opt,name=driver,proto3" json:"driver,omitempty"` } -func (m *ListNetworkOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ListNetworkOptions.Unmarshal(m, b) -} -func (m *ListNetworkOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ListNetworkOptions.Marshal(b, m, deterministic) -} -func (m *ListNetworkOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_ListNetworkOptions.Merge(m, src) +func (x *ListNetworkOptions) Reset() { + *x = ListNetworkOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ListNetworkOptions) XXX_Size() int { - return xxx_messageInfo_ListNetworkOptions.Size(m) + +func (x *ListNetworkOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ListNetworkOptions) XXX_DiscardUnknown() { - xxx_messageInfo_ListNetworkOptions.DiscardUnknown(m) + +func (*ListNetworkOptions) ProtoMessage() {} + +func (x *ListNetworkOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[8] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ListNetworkOptions proto.InternalMessageInfo +// Deprecated: Use ListNetworkOptions.ProtoReflect.Descriptor instead. +func (*ListNetworkOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{8} +} -func (m *ListNetworkOptions) GetPodname() string { - if m != nil { - return m.Podname +func (x *ListNetworkOptions) GetPodname() string { + if x != nil { + return x.Podname } return "" } -func (m *ListNetworkOptions) GetDriver() string { - if m != nil { - return m.Driver +func (x *ListNetworkOptions) GetDriver() string { + if x != nil { + return x.Driver } return "" } type ConnectNetworkOptions struct { - Network string `protobuf:"bytes,1,opt,name=network,proto3" json:"network,omitempty"` - Target string `protobuf:"bytes,2,opt,name=target,proto3" json:"target,omitempty"` - Ipv4 string `protobuf:"bytes,3,opt,name=ipv4,proto3" json:"ipv4,omitempty"` - Ipv6 string `protobuf:"bytes,4,opt,name=ipv6,proto3" json:"ipv6,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ConnectNetworkOptions) Reset() { *m = ConnectNetworkOptions{} } -func (m *ConnectNetworkOptions) String() string { return proto.CompactTextString(m) } -func (*ConnectNetworkOptions) ProtoMessage() {} -func (*ConnectNetworkOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{9} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ConnectNetworkOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ConnectNetworkOptions.Unmarshal(m, b) + Network string `protobuf:"bytes,1,opt,name=network,proto3" json:"network,omitempty"` + Target string `protobuf:"bytes,2,opt,name=target,proto3" json:"target,omitempty"` + Ipv4 string `protobuf:"bytes,3,opt,name=ipv4,proto3" json:"ipv4,omitempty"` + Ipv6 string `protobuf:"bytes,4,opt,name=ipv6,proto3" json:"ipv6,omitempty"` } -func (m *ConnectNetworkOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ConnectNetworkOptions.Marshal(b, m, deterministic) -} -func (m *ConnectNetworkOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_ConnectNetworkOptions.Merge(m, src) + +func (x *ConnectNetworkOptions) Reset() { + *x = ConnectNetworkOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ConnectNetworkOptions) XXX_Size() int { - return xxx_messageInfo_ConnectNetworkOptions.Size(m) + +func (x *ConnectNetworkOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ConnectNetworkOptions) XXX_DiscardUnknown() { - xxx_messageInfo_ConnectNetworkOptions.DiscardUnknown(m) + +func (*ConnectNetworkOptions) ProtoMessage() {} + +func (x *ConnectNetworkOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[9] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ConnectNetworkOptions proto.InternalMessageInfo +// Deprecated: Use ConnectNetworkOptions.ProtoReflect.Descriptor instead. +func (*ConnectNetworkOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{9} +} -func (m *ConnectNetworkOptions) GetNetwork() string { - if m != nil { - return m.Network +func (x *ConnectNetworkOptions) GetNetwork() string { + if x != nil { + return x.Network } return "" } -func (m *ConnectNetworkOptions) GetTarget() string { - if m != nil { - return m.Target +func (x *ConnectNetworkOptions) GetTarget() string { + if x != nil { + return x.Target } return "" } -func (m *ConnectNetworkOptions) GetIpv4() string { - if m != nil { - return m.Ipv4 +func (x *ConnectNetworkOptions) GetIpv4() string { + if x != nil { + return x.Ipv4 } return "" } -func (m *ConnectNetworkOptions) GetIpv6() string { - if m != nil { - return m.Ipv6 +func (x *ConnectNetworkOptions) GetIpv6() string { + if x != nil { + return x.Ipv6 } return "" } type DisconnectNetworkOptions struct { - Network string `protobuf:"bytes,1,opt,name=network,proto3" json:"network,omitempty"` - Target string `protobuf:"bytes,2,opt,name=target,proto3" json:"target,omitempty"` - Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *DisconnectNetworkOptions) Reset() { *m = DisconnectNetworkOptions{} } -func (m *DisconnectNetworkOptions) String() string { return proto.CompactTextString(m) } -func (*DisconnectNetworkOptions) ProtoMessage() {} -func (*DisconnectNetworkOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{10} + Network string `protobuf:"bytes,1,opt,name=network,proto3" json:"network,omitempty"` + Target string `protobuf:"bytes,2,opt,name=target,proto3" json:"target,omitempty"` + Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"` } -func (m *DisconnectNetworkOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_DisconnectNetworkOptions.Unmarshal(m, b) -} -func (m *DisconnectNetworkOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_DisconnectNetworkOptions.Marshal(b, m, deterministic) -} -func (m *DisconnectNetworkOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_DisconnectNetworkOptions.Merge(m, src) +func (x *DisconnectNetworkOptions) Reset() { + *x = DisconnectNetworkOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *DisconnectNetworkOptions) XXX_Size() int { - return xxx_messageInfo_DisconnectNetworkOptions.Size(m) + +func (x *DisconnectNetworkOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *DisconnectNetworkOptions) XXX_DiscardUnknown() { - xxx_messageInfo_DisconnectNetworkOptions.DiscardUnknown(m) + +func (*DisconnectNetworkOptions) ProtoMessage() {} + +func (x *DisconnectNetworkOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[10] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_DisconnectNetworkOptions proto.InternalMessageInfo +// Deprecated: Use DisconnectNetworkOptions.ProtoReflect.Descriptor instead. +func (*DisconnectNetworkOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{10} +} -func (m *DisconnectNetworkOptions) GetNetwork() string { - if m != nil { - return m.Network +func (x *DisconnectNetworkOptions) GetNetwork() string { + if x != nil { + return x.Network } return "" } -func (m *DisconnectNetworkOptions) GetTarget() string { - if m != nil { - return m.Target +func (x *DisconnectNetworkOptions) GetTarget() string { + if x != nil { + return x.Target } return "" } -func (m *DisconnectNetworkOptions) GetForce() bool { - if m != nil { - return m.Force +func (x *DisconnectNetworkOptions) GetForce() bool { + if x != nil { + return x.Force } return false } type Network struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Subnets []string `protobuf:"bytes,2,rep,name=subnets,proto3" json:"subnets,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Network) Reset() { *m = Network{} } -func (m *Network) String() string { return proto.CompactTextString(m) } -func (*Network) ProtoMessage() {} -func (*Network) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{11} + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Subnets []string `protobuf:"bytes,2,rep,name=subnets,proto3" json:"subnets,omitempty"` } -func (m *Network) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Network.Unmarshal(m, b) -} -func (m *Network) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Network.Marshal(b, m, deterministic) -} -func (m *Network) XXX_Merge(src proto.Message) { - xxx_messageInfo_Network.Merge(m, src) +func (x *Network) Reset() { + *x = Network{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Network) XXX_Size() int { - return xxx_messageInfo_Network.Size(m) + +func (x *Network) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Network) XXX_DiscardUnknown() { - xxx_messageInfo_Network.DiscardUnknown(m) + +func (*Network) ProtoMessage() {} + +func (x *Network) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[11] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_Network proto.InternalMessageInfo +// Deprecated: Use Network.ProtoReflect.Descriptor instead. +func (*Network) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{11} +} -func (m *Network) GetName() string { - if m != nil { - return m.Name +func (x *Network) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *Network) GetSubnets() []string { - if m != nil { - return m.Subnets +func (x *Network) GetSubnets() []string { + if x != nil { + return x.Subnets } return nil } type Networks struct { - Networks []*Network `protobuf:"bytes,1,rep,name=networks,proto3" json:"networks,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Networks) Reset() { *m = Networks{} } -func (m *Networks) String() string { return proto.CompactTextString(m) } -func (*Networks) ProtoMessage() {} -func (*Networks) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{12} + Networks []*Network `protobuf:"bytes,1,rep,name=networks,proto3" json:"networks,omitempty"` } -func (m *Networks) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Networks.Unmarshal(m, b) -} -func (m *Networks) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Networks.Marshal(b, m, deterministic) -} -func (m *Networks) XXX_Merge(src proto.Message) { - xxx_messageInfo_Networks.Merge(m, src) +func (x *Networks) Reset() { + *x = Networks{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Networks) XXX_Size() int { - return xxx_messageInfo_Networks.Size(m) + +func (x *Networks) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Networks) XXX_DiscardUnknown() { - xxx_messageInfo_Networks.DiscardUnknown(m) + +func (*Networks) ProtoMessage() {} + +func (x *Networks) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_Networks proto.InternalMessageInfo +// Deprecated: Use Networks.ProtoReflect.Descriptor instead. +func (*Networks) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{12} +} -func (m *Networks) GetNetworks() []*Network { - if m != nil { - return m.Networks +func (x *Networks) GetNetworks() []*Network { + if x != nil { + return x.Networks } return nil } type Node struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Endpoint string `protobuf:"bytes,2,opt,name=endpoint,proto3" json:"endpoint,omitempty"` - Podname string `protobuf:"bytes,3,opt,name=podname,proto3" json:"podname,omitempty"` - Cpu map[string]int32 `protobuf:"bytes,4,rep,name=cpu,proto3" json:"cpu,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - CpuUsed float64 `protobuf:"fixed64,5,opt,name=cpu_used,json=cpuUsed,proto3" json:"cpu_used,omitempty"` - Memory int64 `protobuf:"varint,6,opt,name=memory,proto3" json:"memory,omitempty"` - MemoryUsed int64 `protobuf:"varint,7,opt,name=memory_used,json=memoryUsed,proto3" json:"memory_used,omitempty"` - Available bool `protobuf:"varint,8,opt,name=available,proto3" json:"available,omitempty"` - Labels map[string]string `protobuf:"bytes,9,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - InitMemory int64 `protobuf:"varint,10,opt,name=init_memory,json=initMemory,proto3" json:"init_memory,omitempty"` - InitCpu map[string]int32 `protobuf:"bytes,11,rep,name=init_cpu,json=initCpu,proto3" json:"init_cpu,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Info string `protobuf:"bytes,12,opt,name=info,proto3" json:"info,omitempty"` - Numa map[string]string `protobuf:"bytes,13,rep,name=numa,proto3" json:"numa,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - NumaMemory map[string]int64 `protobuf:"bytes,14,rep,name=numa_memory,json=numaMemory,proto3" json:"numa_memory,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Storage int64 `protobuf:"varint,15,opt,name=storage,proto3" json:"storage,omitempty"` - StorageUsed int64 `protobuf:"varint,16,opt,name=storage_used,json=storageUsed,proto3" json:"storage_used,omitempty"` - InitStorage int64 `protobuf:"varint,17,opt,name=init_storage,json=initStorage,proto3" json:"init_storage,omitempty"` - InitVolume map[string]int64 `protobuf:"bytes,18,rep,name=init_volume,json=initVolume,proto3" json:"init_volume,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Volume map[string]int64 `protobuf:"bytes,19,rep,name=volume,proto3" json:"volume,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - VolumeUsed int64 `protobuf:"varint,20,opt,name=volume_used,json=volumeUsed,proto3" json:"volume_used,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *Node) Reset() { *m = Node{} } -func (m *Node) String() string { return proto.CompactTextString(m) } -func (*Node) ProtoMessage() {} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Endpoint string `protobuf:"bytes,2,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + Podname string `protobuf:"bytes,3,opt,name=podname,proto3" json:"podname,omitempty"` + Cpu map[string]int32 `protobuf:"bytes,4,rep,name=cpu,proto3" json:"cpu,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + CpuUsed float64 `protobuf:"fixed64,5,opt,name=cpu_used,json=cpuUsed,proto3" json:"cpu_used,omitempty"` + Memory int64 `protobuf:"varint,6,opt,name=memory,proto3" json:"memory,omitempty"` + MemoryUsed int64 `protobuf:"varint,7,opt,name=memory_used,json=memoryUsed,proto3" json:"memory_used,omitempty"` + Available bool `protobuf:"varint,8,opt,name=available,proto3" json:"available,omitempty"` + Labels map[string]string `protobuf:"bytes,9,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + InitMemory int64 `protobuf:"varint,10,opt,name=init_memory,json=initMemory,proto3" json:"init_memory,omitempty"` + InitCpu map[string]int32 `protobuf:"bytes,11,rep,name=init_cpu,json=initCpu,proto3" json:"init_cpu,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Info string `protobuf:"bytes,12,opt,name=info,proto3" json:"info,omitempty"` + Numa map[string]string `protobuf:"bytes,13,rep,name=numa,proto3" json:"numa,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + NumaMemory map[string]int64 `protobuf:"bytes,14,rep,name=numa_memory,json=numaMemory,proto3" json:"numa_memory,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Storage int64 `protobuf:"varint,15,opt,name=storage,proto3" json:"storage,omitempty"` + StorageUsed int64 `protobuf:"varint,16,opt,name=storage_used,json=storageUsed,proto3" json:"storage_used,omitempty"` + InitStorage int64 `protobuf:"varint,17,opt,name=init_storage,json=initStorage,proto3" json:"init_storage,omitempty"` + InitVolume map[string]int64 `protobuf:"bytes,18,rep,name=init_volume,json=initVolume,proto3" json:"init_volume,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Volume map[string]int64 `protobuf:"bytes,19,rep,name=volume,proto3" json:"volume,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + VolumeUsed int64 `protobuf:"varint,20,opt,name=volume_used,json=volumeUsed,proto3" json:"volume_used,omitempty"` +} + +func (x *Node) Reset() { + *x = Node{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Node) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Node) ProtoMessage() {} + +func (x *Node) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Node.ProtoReflect.Descriptor instead. func (*Node) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{13} -} - -func (m *Node) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Node.Unmarshal(m, b) -} -func (m *Node) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Node.Marshal(b, m, deterministic) -} -func (m *Node) XXX_Merge(src proto.Message) { - xxx_messageInfo_Node.Merge(m, src) -} -func (m *Node) XXX_Size() int { - return xxx_messageInfo_Node.Size(m) -} -func (m *Node) XXX_DiscardUnknown() { - xxx_messageInfo_Node.DiscardUnknown(m) + return file_core_proto_rawDescGZIP(), []int{13} } -var xxx_messageInfo_Node proto.InternalMessageInfo - -func (m *Node) GetName() string { - if m != nil { - return m.Name +func (x *Node) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *Node) GetEndpoint() string { - if m != nil { - return m.Endpoint +func (x *Node) GetEndpoint() string { + if x != nil { + return x.Endpoint } return "" } -func (m *Node) GetPodname() string { - if m != nil { - return m.Podname +func (x *Node) GetPodname() string { + if x != nil { + return x.Podname } return "" } -func (m *Node) GetCpu() map[string]int32 { - if m != nil { - return m.Cpu +func (x *Node) GetCpu() map[string]int32 { + if x != nil { + return x.Cpu } return nil } -func (m *Node) GetCpuUsed() float64 { - if m != nil { - return m.CpuUsed +func (x *Node) GetCpuUsed() float64 { + if x != nil { + return x.CpuUsed } return 0 } -func (m *Node) GetMemory() int64 { - if m != nil { - return m.Memory +func (x *Node) GetMemory() int64 { + if x != nil { + return x.Memory } return 0 } -func (m *Node) GetMemoryUsed() int64 { - if m != nil { - return m.MemoryUsed +func (x *Node) GetMemoryUsed() int64 { + if x != nil { + return x.MemoryUsed } return 0 } -func (m *Node) GetAvailable() bool { - if m != nil { - return m.Available +func (x *Node) GetAvailable() bool { + if x != nil { + return x.Available } return false } -func (m *Node) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *Node) GetLabels() map[string]string { + if x != nil { + return x.Labels } return nil } -func (m *Node) GetInitMemory() int64 { - if m != nil { - return m.InitMemory +func (x *Node) GetInitMemory() int64 { + if x != nil { + return x.InitMemory } return 0 } -func (m *Node) GetInitCpu() map[string]int32 { - if m != nil { - return m.InitCpu +func (x *Node) GetInitCpu() map[string]int32 { + if x != nil { + return x.InitCpu } return nil } -func (m *Node) GetInfo() string { - if m != nil { - return m.Info +func (x *Node) GetInfo() string { + if x != nil { + return x.Info } return "" } -func (m *Node) GetNuma() map[string]string { - if m != nil { - return m.Numa +func (x *Node) GetNuma() map[string]string { + if x != nil { + return x.Numa } return nil } -func (m *Node) GetNumaMemory() map[string]int64 { - if m != nil { - return m.NumaMemory +func (x *Node) GetNumaMemory() map[string]int64 { + if x != nil { + return x.NumaMemory } return nil } -func (m *Node) GetStorage() int64 { - if m != nil { - return m.Storage +func (x *Node) GetStorage() int64 { + if x != nil { + return x.Storage } return 0 } -func (m *Node) GetStorageUsed() int64 { - if m != nil { - return m.StorageUsed +func (x *Node) GetStorageUsed() int64 { + if x != nil { + return x.StorageUsed } return 0 } -func (m *Node) GetInitStorage() int64 { - if m != nil { - return m.InitStorage +func (x *Node) GetInitStorage() int64 { + if x != nil { + return x.InitStorage } return 0 } -func (m *Node) GetInitVolume() map[string]int64 { - if m != nil { - return m.InitVolume +func (x *Node) GetInitVolume() map[string]int64 { + if x != nil { + return x.InitVolume } return nil } -func (m *Node) GetVolume() map[string]int64 { - if m != nil { - return m.Volume +func (x *Node) GetVolume() map[string]int64 { + if x != nil { + return x.Volume } return nil } -func (m *Node) GetVolumeUsed() int64 { - if m != nil { - return m.VolumeUsed +func (x *Node) GetVolumeUsed() int64 { + if x != nil { + return x.VolumeUsed } return 0 } type Nodes struct { - Nodes []*Node `protobuf:"bytes,1,rep,name=nodes,proto3" json:"nodes,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Nodes) Reset() { *m = Nodes{} } -func (m *Nodes) String() string { return proto.CompactTextString(m) } -func (*Nodes) ProtoMessage() {} -func (*Nodes) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{14} + Nodes []*Node `protobuf:"bytes,1,rep,name=nodes,proto3" json:"nodes,omitempty"` } -func (m *Nodes) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Nodes.Unmarshal(m, b) -} -func (m *Nodes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Nodes.Marshal(b, m, deterministic) -} -func (m *Nodes) XXX_Merge(src proto.Message) { - xxx_messageInfo_Nodes.Merge(m, src) +func (x *Nodes) Reset() { + *x = Nodes{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Nodes) XXX_Size() int { - return xxx_messageInfo_Nodes.Size(m) + +func (x *Nodes) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Nodes) XXX_DiscardUnknown() { - xxx_messageInfo_Nodes.DiscardUnknown(m) + +func (*Nodes) ProtoMessage() {} + +func (x *Nodes) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_Nodes proto.InternalMessageInfo +// Deprecated: Use Nodes.ProtoReflect.Descriptor instead. +func (*Nodes) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{14} +} -func (m *Nodes) GetNodes() []*Node { - if m != nil { - return m.Nodes +func (x *Nodes) GetNodes() []*Node { + if x != nil { + return x.Nodes } return nil } type NodeAvailable struct { - Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` - Podname string `protobuf:"bytes,2,opt,name=podname,proto3" json:"podname,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *NodeAvailable) Reset() { *m = NodeAvailable{} } -func (m *NodeAvailable) String() string { return proto.CompactTextString(m) } -func (*NodeAvailable) ProtoMessage() {} -func (*NodeAvailable) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{15} + Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` + Podname string `protobuf:"bytes,2,opt,name=podname,proto3" json:"podname,omitempty"` } -func (m *NodeAvailable) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_NodeAvailable.Unmarshal(m, b) -} -func (m *NodeAvailable) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_NodeAvailable.Marshal(b, m, deterministic) -} -func (m *NodeAvailable) XXX_Merge(src proto.Message) { - xxx_messageInfo_NodeAvailable.Merge(m, src) +func (x *NodeAvailable) Reset() { + *x = NodeAvailable{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *NodeAvailable) XXX_Size() int { - return xxx_messageInfo_NodeAvailable.Size(m) + +func (x *NodeAvailable) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *NodeAvailable) XXX_DiscardUnknown() { - xxx_messageInfo_NodeAvailable.DiscardUnknown(m) + +func (*NodeAvailable) ProtoMessage() {} + +func (x *NodeAvailable) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_NodeAvailable proto.InternalMessageInfo +// Deprecated: Use NodeAvailable.ProtoReflect.Descriptor instead. +func (*NodeAvailable) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{15} +} -func (m *NodeAvailable) GetNodename() string { - if m != nil { - return m.Nodename +func (x *NodeAvailable) GetNodename() string { + if x != nil { + return x.Nodename } return "" } -func (m *NodeAvailable) GetPodname() string { - if m != nil { - return m.Podname +func (x *NodeAvailable) GetPodname() string { + if x != nil { + return x.Podname } return "" } type SetNodeOptions struct { - Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` - StatusOpt TriOpt `protobuf:"varint,2,opt,name=status_opt,json=statusOpt,proto3,enum=pb.TriOpt" json:"status_opt,omitempty"` - DeltaCpu map[string]int32 `protobuf:"bytes,3,rep,name=delta_cpu,json=deltaCpu,proto3" json:"delta_cpu,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - DeltaMemory int64 `protobuf:"varint,4,opt,name=delta_memory,json=deltaMemory,proto3" json:"delta_memory,omitempty"` - DeltaStorage int64 `protobuf:"varint,5,opt,name=delta_storage,json=deltaStorage,proto3" json:"delta_storage,omitempty"` - DeltaNumaMemory map[string]int64 `protobuf:"bytes,6,rep,name=delta_numa_memory,json=deltaNumaMemory,proto3" json:"delta_numa_memory,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Numa map[string]string `protobuf:"bytes,7,rep,name=numa,proto3" json:"numa,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Labels map[string]string `protobuf:"bytes,8,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - DeltaVolume map[string]int64 `protobuf:"bytes,9,rep,name=delta_volume,json=deltaVolume,proto3" json:"delta_volume,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - ContainersDown bool `protobuf:"varint,10,opt,name=containers_down,json=containersDown,proto3" json:"containers_down,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SetNodeOptions) Reset() { *m = SetNodeOptions{} } -func (m *SetNodeOptions) String() string { return proto.CompactTextString(m) } -func (*SetNodeOptions) ProtoMessage() {} -func (*SetNodeOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{16} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *SetNodeOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SetNodeOptions.Unmarshal(m, b) -} -func (m *SetNodeOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SetNodeOptions.Marshal(b, m, deterministic) + Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` + StatusOpt TriOpt `protobuf:"varint,2,opt,name=status_opt,json=statusOpt,proto3,enum=pb.TriOpt" json:"status_opt,omitempty"` + DeltaCpu map[string]int32 `protobuf:"bytes,3,rep,name=delta_cpu,json=deltaCpu,proto3" json:"delta_cpu,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + DeltaMemory int64 `protobuf:"varint,4,opt,name=delta_memory,json=deltaMemory,proto3" json:"delta_memory,omitempty"` + DeltaStorage int64 `protobuf:"varint,5,opt,name=delta_storage,json=deltaStorage,proto3" json:"delta_storage,omitempty"` + DeltaNumaMemory map[string]int64 `protobuf:"bytes,6,rep,name=delta_numa_memory,json=deltaNumaMemory,proto3" json:"delta_numa_memory,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Numa map[string]string `protobuf:"bytes,7,rep,name=numa,proto3" json:"numa,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Labels map[string]string `protobuf:"bytes,8,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + DeltaVolume map[string]int64 `protobuf:"bytes,9,rep,name=delta_volume,json=deltaVolume,proto3" json:"delta_volume,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + ContainersDown bool `protobuf:"varint,10,opt,name=containers_down,json=containersDown,proto3" json:"containers_down,omitempty"` } -func (m *SetNodeOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_SetNodeOptions.Merge(m, src) + +func (x *SetNodeOptions) Reset() { + *x = SetNodeOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *SetNodeOptions) XXX_Size() int { - return xxx_messageInfo_SetNodeOptions.Size(m) + +func (x *SetNodeOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *SetNodeOptions) XXX_DiscardUnknown() { - xxx_messageInfo_SetNodeOptions.DiscardUnknown(m) + +func (*SetNodeOptions) ProtoMessage() {} + +func (x *SetNodeOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_SetNodeOptions proto.InternalMessageInfo +// Deprecated: Use SetNodeOptions.ProtoReflect.Descriptor instead. +func (*SetNodeOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{16} +} -func (m *SetNodeOptions) GetNodename() string { - if m != nil { - return m.Nodename +func (x *SetNodeOptions) GetNodename() string { + if x != nil { + return x.Nodename } return "" } -func (m *SetNodeOptions) GetStatusOpt() TriOpt { - if m != nil { - return m.StatusOpt +func (x *SetNodeOptions) GetStatusOpt() TriOpt { + if x != nil { + return x.StatusOpt } return TriOpt_KEEP } -func (m *SetNodeOptions) GetDeltaCpu() map[string]int32 { - if m != nil { - return m.DeltaCpu +func (x *SetNodeOptions) GetDeltaCpu() map[string]int32 { + if x != nil { + return x.DeltaCpu } return nil } -func (m *SetNodeOptions) GetDeltaMemory() int64 { - if m != nil { - return m.DeltaMemory +func (x *SetNodeOptions) GetDeltaMemory() int64 { + if x != nil { + return x.DeltaMemory } return 0 } -func (m *SetNodeOptions) GetDeltaStorage() int64 { - if m != nil { - return m.DeltaStorage +func (x *SetNodeOptions) GetDeltaStorage() int64 { + if x != nil { + return x.DeltaStorage } return 0 } -func (m *SetNodeOptions) GetDeltaNumaMemory() map[string]int64 { - if m != nil { - return m.DeltaNumaMemory +func (x *SetNodeOptions) GetDeltaNumaMemory() map[string]int64 { + if x != nil { + return x.DeltaNumaMemory } return nil } -func (m *SetNodeOptions) GetNuma() map[string]string { - if m != nil { - return m.Numa +func (x *SetNodeOptions) GetNuma() map[string]string { + if x != nil { + return x.Numa } return nil } -func (m *SetNodeOptions) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *SetNodeOptions) GetLabels() map[string]string { + if x != nil { + return x.Labels } return nil } -func (m *SetNodeOptions) GetDeltaVolume() map[string]int64 { - if m != nil { - return m.DeltaVolume +func (x *SetNodeOptions) GetDeltaVolume() map[string]int64 { + if x != nil { + return x.DeltaVolume } return nil } -func (m *SetNodeOptions) GetContainersDown() bool { - if m != nil { - return m.ContainersDown +func (x *SetNodeOptions) GetContainersDown() bool { + if x != nil { + return x.ContainersDown } return false } type Container struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Podname string `protobuf:"bytes,2,opt,name=podname,proto3" json:"podname,omitempty"` - Nodename string `protobuf:"bytes,3,opt,name=nodename,proto3" json:"nodename,omitempty"` - Name string `protobuf:"bytes,4,opt,name=name,proto3" json:"name,omitempty"` - Cpu map[string]int32 `protobuf:"bytes,5,rep,name=cpu,proto3" json:"cpu,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Quota float64 `protobuf:"fixed64,6,opt,name=quota,proto3" json:"quota,omitempty"` - Memory int64 `protobuf:"varint,7,opt,name=memory,proto3" json:"memory,omitempty"` - Privileged bool `protobuf:"varint,8,opt,name=privileged,proto3" json:"privileged,omitempty"` - Labels map[string]string `protobuf:"bytes,9,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Publish map[string]string `protobuf:"bytes,10,rep,name=publish,proto3" json:"publish,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Image string `protobuf:"bytes,11,opt,name=image,proto3" json:"image,omitempty"` - Storage int64 `protobuf:"varint,12,opt,name=storage,proto3" json:"storage,omitempty"` - Status *ContainerStatus `protobuf:"bytes,13,opt,name=status,proto3" json:"status,omitempty"` - Volumes []string `protobuf:"bytes,14,rep,name=volumes,proto3" json:"volumes,omitempty"` - VolumePlan map[string]*Volume `protobuf:"bytes,15,rep,name=volume_plan,json=volumePlan,proto3" json:"volume_plan,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - QuotaRequest float64 `protobuf:"fixed64,16,opt,name=quota_request,json=quotaRequest,proto3" json:"quota_request,omitempty"` - MemoryRequest int64 `protobuf:"varint,17,opt,name=memory_request,json=memoryRequest,proto3" json:"memory_request,omitempty"` - StorageRequest int64 `protobuf:"varint,18,opt,name=storage_request,json=storageRequest,proto3" json:"storage_request,omitempty"` - VolumesRequest []string `protobuf:"bytes,19,rep,name=volumes_request,json=volumesRequest,proto3" json:"volumes_request,omitempty"` - VolumePlanRequest map[string]*Volume `protobuf:"bytes,20,rep,name=volume_plan_request,json=volumePlanRequest,proto3" json:"volume_plan_request,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *Container) Reset() { *m = Container{} } -func (m *Container) String() string { return proto.CompactTextString(m) } -func (*Container) ProtoMessage() {} -func (*Container) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{17} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Container) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Container.Unmarshal(m, b) + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Podname string `protobuf:"bytes,2,opt,name=podname,proto3" json:"podname,omitempty"` + Nodename string `protobuf:"bytes,3,opt,name=nodename,proto3" json:"nodename,omitempty"` + Name string `protobuf:"bytes,4,opt,name=name,proto3" json:"name,omitempty"` + Privileged bool `protobuf:"varint,5,opt,name=privileged,proto3" json:"privileged,omitempty"` + Labels map[string]string `protobuf:"bytes,6,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Publish map[string]string `protobuf:"bytes,7,rep,name=publish,proto3" json:"publish,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Image string `protobuf:"bytes,8,opt,name=image,proto3" json:"image,omitempty"` + Status *ContainerStatus `protobuf:"bytes,9,opt,name=status,proto3" json:"status,omitempty"` + Resource *Resource `protobuf:"bytes,10,opt,name=resource,proto3" json:"resource,omitempty"` } -func (m *Container) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Container.Marshal(b, m, deterministic) -} -func (m *Container) XXX_Merge(src proto.Message) { - xxx_messageInfo_Container.Merge(m, src) + +func (x *Container) Reset() { + *x = Container{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Container) XXX_Size() int { - return xxx_messageInfo_Container.Size(m) + +func (x *Container) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Container) XXX_DiscardUnknown() { - xxx_messageInfo_Container.DiscardUnknown(m) + +func (*Container) ProtoMessage() {} + +func (x *Container) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[17] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_Container proto.InternalMessageInfo +// Deprecated: Use Container.ProtoReflect.Descriptor instead. +func (*Container) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{17} +} -func (m *Container) GetId() string { - if m != nil { - return m.Id +func (x *Container) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *Container) GetPodname() string { - if m != nil { - return m.Podname +func (x *Container) GetPodname() string { + if x != nil { + return x.Podname } return "" } -func (m *Container) GetNodename() string { - if m != nil { - return m.Nodename +func (x *Container) GetNodename() string { + if x != nil { + return x.Nodename } return "" } -func (m *Container) GetName() string { - if m != nil { - return m.Name +func (x *Container) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *Container) GetCpu() map[string]int32 { - if m != nil { - return m.Cpu +func (x *Container) GetPrivileged() bool { + if x != nil { + return x.Privileged } - return nil + return false } -func (m *Container) GetQuota() float64 { - if m != nil { - return m.Quota +func (x *Container) GetLabels() map[string]string { + if x != nil { + return x.Labels } - return 0 + return nil } -func (m *Container) GetMemory() int64 { - if m != nil { - return m.Memory +func (x *Container) GetPublish() map[string]string { + if x != nil { + return x.Publish } - return 0 + return nil } -func (m *Container) GetPrivileged() bool { - if m != nil { - return m.Privileged +func (x *Container) GetImage() string { + if x != nil { + return x.Image } - return false + return "" } -func (m *Container) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *Container) GetStatus() *ContainerStatus { + if x != nil { + return x.Status } return nil } -func (m *Container) GetPublish() map[string]string { - if m != nil { - return m.Publish +func (x *Container) GetResource() *Resource { + if x != nil { + return x.Resource } return nil } -func (m *Container) GetImage() string { - if m != nil { - return m.Image - } - return "" +type ContainerStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Running bool `protobuf:"varint,2,opt,name=running,proto3" json:"running,omitempty"` + Healthy bool `protobuf:"varint,3,opt,name=healthy,proto3" json:"healthy,omitempty"` + Networks map[string]string `protobuf:"bytes,4,rep,name=networks,proto3" json:"networks,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Extension []byte `protobuf:"bytes,5,opt,name=extension,proto3" json:"extension,omitempty"` + Ttl int64 `protobuf:"varint,6,opt,name=ttl,proto3" json:"ttl,omitempty"` } -func (m *Container) GetStorage() int64 { - if m != nil { - return m.Storage +func (x *ContainerStatus) Reset() { + *x = ContainerStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } - return 0 } -func (m *Container) GetStatus() *ContainerStatus { - if m != nil { - return m.Status - } - return nil +func (x *ContainerStatus) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Container) GetVolumes() []string { - if m != nil { - return m.Volumes +func (*ContainerStatus) ProtoMessage() {} + +func (x *ContainerStatus) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[18] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } - return nil + return mi.MessageOf(x) } -func (m *Container) GetVolumePlan() map[string]*Volume { - if m != nil { - return m.VolumePlan - } - return nil +// Deprecated: Use ContainerStatus.ProtoReflect.Descriptor instead. +func (*ContainerStatus) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{18} } -func (m *Container) GetQuotaRequest() float64 { - if m != nil { - return m.QuotaRequest +func (x *ContainerStatus) GetId() string { + if x != nil { + return x.Id } - return 0 + return "" } -func (m *Container) GetMemoryRequest() int64 { - if m != nil { - return m.MemoryRequest +func (x *ContainerStatus) GetRunning() bool { + if x != nil { + return x.Running } - return 0 + return false } -func (m *Container) GetStorageRequest() int64 { - if m != nil { - return m.StorageRequest +func (x *ContainerStatus) GetHealthy() bool { + if x != nil { + return x.Healthy } - return 0 + return false } -func (m *Container) GetVolumesRequest() []string { - if m != nil { - return m.VolumesRequest +func (x *ContainerStatus) GetNetworks() map[string]string { + if x != nil { + return x.Networks } return nil } -func (m *Container) GetVolumePlanRequest() map[string]*Volume { - if m != nil { - return m.VolumePlanRequest +func (x *ContainerStatus) GetExtension() []byte { + if x != nil { + return x.Extension } return nil } -type ContainerStatus struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Running bool `protobuf:"varint,2,opt,name=running,proto3" json:"running,omitempty"` - Healthy bool `protobuf:"varint,3,opt,name=healthy,proto3" json:"healthy,omitempty"` - Networks map[string]string `protobuf:"bytes,4,rep,name=networks,proto3" json:"networks,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Extension []byte `protobuf:"bytes,5,opt,name=extension,proto3" json:"extension,omitempty"` - Ttl int64 `protobuf:"varint,6,opt,name=ttl,proto3" json:"ttl,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ContainerStatus) Reset() { *m = ContainerStatus{} } -func (m *ContainerStatus) String() string { return proto.CompactTextString(m) } -func (*ContainerStatus) ProtoMessage() {} -func (*ContainerStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{18} +func (x *ContainerStatus) GetTtl() int64 { + if x != nil { + return x.Ttl + } + return 0 } -func (m *ContainerStatus) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ContainerStatus.Unmarshal(m, b) -} -func (m *ContainerStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ContainerStatus.Marshal(b, m, deterministic) -} -func (m *ContainerStatus) XXX_Merge(src proto.Message) { - xxx_messageInfo_ContainerStatus.Merge(m, src) -} -func (m *ContainerStatus) XXX_Size() int { - return xxx_messageInfo_ContainerStatus.Size(m) -} -func (m *ContainerStatus) XXX_DiscardUnknown() { - xxx_messageInfo_ContainerStatus.DiscardUnknown(m) -} +type ContainersStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -var xxx_messageInfo_ContainerStatus proto.InternalMessageInfo + Status []*ContainerStatus `protobuf:"bytes,1,rep,name=status,proto3" json:"status,omitempty"` +} -func (m *ContainerStatus) GetId() string { - if m != nil { - return m.Id +func (x *ContainersStatus) Reset() { + *x = ContainersStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } - return "" } -func (m *ContainerStatus) GetRunning() bool { - if m != nil { - return m.Running - } - return false +func (x *ContainersStatus) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ContainerStatus) GetHealthy() bool { - if m != nil { - return m.Healthy +func (*ContainersStatus) ProtoMessage() {} + +func (x *ContainersStatus) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[19] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } - return false + return mi.MessageOf(x) } -func (m *ContainerStatus) GetNetworks() map[string]string { - if m != nil { - return m.Networks - } - return nil +// Deprecated: Use ContainersStatus.ProtoReflect.Descriptor instead. +func (*ContainersStatus) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{19} } -func (m *ContainerStatus) GetExtension() []byte { - if m != nil { - return m.Extension +func (x *ContainersStatus) GetStatus() []*ContainerStatus { + if x != nil { + return x.Status } return nil } -func (m *ContainerStatus) GetTtl() int64 { - if m != nil { - return m.Ttl - } - return 0 -} +type ContainerStatusStreamMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -type ContainersStatus struct { - Status []*ContainerStatus `protobuf:"bytes,1,rep,name=status,proto3" json:"status,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Container *Container `protobuf:"bytes,2,opt,name=container,proto3" json:"container,omitempty"` + Status *ContainerStatus `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"` + Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"` + Delete bool `protobuf:"varint,5,opt,name=delete,proto3" json:"delete,omitempty"` } -func (m *ContainersStatus) Reset() { *m = ContainersStatus{} } -func (m *ContainersStatus) String() string { return proto.CompactTextString(m) } -func (*ContainersStatus) ProtoMessage() {} -func (*ContainersStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{19} +func (x *ContainerStatusStreamMessage) Reset() { + *x = ContainerStatusStreamMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ContainersStatus) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ContainersStatus.Unmarshal(m, b) -} -func (m *ContainersStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ContainersStatus.Marshal(b, m, deterministic) -} -func (m *ContainersStatus) XXX_Merge(src proto.Message) { - xxx_messageInfo_ContainersStatus.Merge(m, src) -} -func (m *ContainersStatus) XXX_Size() int { - return xxx_messageInfo_ContainersStatus.Size(m) -} -func (m *ContainersStatus) XXX_DiscardUnknown() { - xxx_messageInfo_ContainersStatus.DiscardUnknown(m) +func (x *ContainerStatusStreamMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -var xxx_messageInfo_ContainersStatus proto.InternalMessageInfo +func (*ContainerStatusStreamMessage) ProtoMessage() {} -func (m *ContainersStatus) GetStatus() []*ContainerStatus { - if m != nil { - return m.Status +func (x *ContainerStatusStreamMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[20] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } - return nil + return mi.MessageOf(x) } -type ContainerStatusStreamMessage struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Container *Container `protobuf:"bytes,2,opt,name=container,proto3" json:"container,omitempty"` - Status *ContainerStatus `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"` - Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"` - Delete bool `protobuf:"varint,5,opt,name=delete,proto3" json:"delete,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ContainerStatusStreamMessage) Reset() { *m = ContainerStatusStreamMessage{} } -func (m *ContainerStatusStreamMessage) String() string { return proto.CompactTextString(m) } -func (*ContainerStatusStreamMessage) ProtoMessage() {} +// Deprecated: Use ContainerStatusStreamMessage.ProtoReflect.Descriptor instead. func (*ContainerStatusStreamMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{20} -} - -func (m *ContainerStatusStreamMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ContainerStatusStreamMessage.Unmarshal(m, b) -} -func (m *ContainerStatusStreamMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ContainerStatusStreamMessage.Marshal(b, m, deterministic) -} -func (m *ContainerStatusStreamMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_ContainerStatusStreamMessage.Merge(m, src) -} -func (m *ContainerStatusStreamMessage) XXX_Size() int { - return xxx_messageInfo_ContainerStatusStreamMessage.Size(m) + return file_core_proto_rawDescGZIP(), []int{20} } -func (m *ContainerStatusStreamMessage) XXX_DiscardUnknown() { - xxx_messageInfo_ContainerStatusStreamMessage.DiscardUnknown(m) -} - -var xxx_messageInfo_ContainerStatusStreamMessage proto.InternalMessageInfo -func (m *ContainerStatusStreamMessage) GetId() string { - if m != nil { - return m.Id +func (x *ContainerStatusStreamMessage) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *ContainerStatusStreamMessage) GetContainer() *Container { - if m != nil { - return m.Container +func (x *ContainerStatusStreamMessage) GetContainer() *Container { + if x != nil { + return x.Container } return nil } -func (m *ContainerStatusStreamMessage) GetStatus() *ContainerStatus { - if m != nil { - return m.Status +func (x *ContainerStatusStreamMessage) GetStatus() *ContainerStatus { + if x != nil { + return x.Status } return nil } -func (m *ContainerStatusStreamMessage) GetError() string { - if m != nil { - return m.Error +func (x *ContainerStatusStreamMessage) GetError() string { + if x != nil { + return x.Error } return "" } -func (m *ContainerStatusStreamMessage) GetDelete() bool { - if m != nil { - return m.Delete +func (x *ContainerStatusStreamMessage) GetDelete() bool { + if x != nil { + return x.Delete } return false } type SetContainersStatusOptions struct { - Status []*ContainerStatus `protobuf:"bytes,1,rep,name=status,proto3" json:"status,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *SetContainersStatusOptions) Reset() { *m = SetContainersStatusOptions{} } -func (m *SetContainersStatusOptions) String() string { return proto.CompactTextString(m) } -func (*SetContainersStatusOptions) ProtoMessage() {} -func (*SetContainersStatusOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{21} + Status []*ContainerStatus `protobuf:"bytes,1,rep,name=status,proto3" json:"status,omitempty"` } -func (m *SetContainersStatusOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SetContainersStatusOptions.Unmarshal(m, b) -} -func (m *SetContainersStatusOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SetContainersStatusOptions.Marshal(b, m, deterministic) -} -func (m *SetContainersStatusOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_SetContainersStatusOptions.Merge(m, src) +func (x *SetContainersStatusOptions) Reset() { + *x = SetContainersStatusOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *SetContainersStatusOptions) XXX_Size() int { - return xxx_messageInfo_SetContainersStatusOptions.Size(m) + +func (x *SetContainersStatusOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *SetContainersStatusOptions) XXX_DiscardUnknown() { - xxx_messageInfo_SetContainersStatusOptions.DiscardUnknown(m) + +func (*SetContainersStatusOptions) ProtoMessage() {} + +func (x *SetContainersStatusOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[21] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_SetContainersStatusOptions proto.InternalMessageInfo +// Deprecated: Use SetContainersStatusOptions.ProtoReflect.Descriptor instead. +func (*SetContainersStatusOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{21} +} -func (m *SetContainersStatusOptions) GetStatus() []*ContainerStatus { - if m != nil { - return m.Status +func (x *SetContainersStatusOptions) GetStatus() []*ContainerStatus { + if x != nil { + return x.Status } return nil } type ContainerStatusStreamOptions struct { - Appname string `protobuf:"bytes,1,opt,name=appname,proto3" json:"appname,omitempty"` - Entrypoint string `protobuf:"bytes,2,opt,name=entrypoint,proto3" json:"entrypoint,omitempty"` - Nodename string `protobuf:"bytes,3,opt,name=nodename,proto3" json:"nodename,omitempty"` - Labels map[string]string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ContainerStatusStreamOptions) Reset() { *m = ContainerStatusStreamOptions{} } -func (m *ContainerStatusStreamOptions) String() string { return proto.CompactTextString(m) } -func (*ContainerStatusStreamOptions) ProtoMessage() {} -func (*ContainerStatusStreamOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{22} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ContainerStatusStreamOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ContainerStatusStreamOptions.Unmarshal(m, b) -} -func (m *ContainerStatusStreamOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ContainerStatusStreamOptions.Marshal(b, m, deterministic) + Appname string `protobuf:"bytes,1,opt,name=appname,proto3" json:"appname,omitempty"` + Entrypoint string `protobuf:"bytes,2,opt,name=entrypoint,proto3" json:"entrypoint,omitempty"` + Nodename string `protobuf:"bytes,3,opt,name=nodename,proto3" json:"nodename,omitempty"` + Labels map[string]string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (m *ContainerStatusStreamOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_ContainerStatusStreamOptions.Merge(m, src) + +func (x *ContainerStatusStreamOptions) Reset() { + *x = ContainerStatusStreamOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ContainerStatusStreamOptions) XXX_Size() int { - return xxx_messageInfo_ContainerStatusStreamOptions.Size(m) + +func (x *ContainerStatusStreamOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ContainerStatusStreamOptions) XXX_DiscardUnknown() { - xxx_messageInfo_ContainerStatusStreamOptions.DiscardUnknown(m) + +func (*ContainerStatusStreamOptions) ProtoMessage() {} + +func (x *ContainerStatusStreamOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[22] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ContainerStatusStreamOptions proto.InternalMessageInfo +// Deprecated: Use ContainerStatusStreamOptions.ProtoReflect.Descriptor instead. +func (*ContainerStatusStreamOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{22} +} -func (m *ContainerStatusStreamOptions) GetAppname() string { - if m != nil { - return m.Appname +func (x *ContainerStatusStreamOptions) GetAppname() string { + if x != nil { + return x.Appname } return "" } -func (m *ContainerStatusStreamOptions) GetEntrypoint() string { - if m != nil { - return m.Entrypoint +func (x *ContainerStatusStreamOptions) GetEntrypoint() string { + if x != nil { + return x.Entrypoint } return "" } -func (m *ContainerStatusStreamOptions) GetNodename() string { - if m != nil { - return m.Nodename +func (x *ContainerStatusStreamOptions) GetNodename() string { + if x != nil { + return x.Nodename } return "" } -func (m *ContainerStatusStreamOptions) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *ContainerStatusStreamOptions) GetLabels() map[string]string { + if x != nil { + return x.Labels } return nil } type Containers struct { - Containers []*Container `protobuf:"bytes,1,rep,name=containers,proto3" json:"containers,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Containers) Reset() { *m = Containers{} } -func (m *Containers) String() string { return proto.CompactTextString(m) } -func (*Containers) ProtoMessage() {} -func (*Containers) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{23} + Containers []*Container `protobuf:"bytes,1,rep,name=containers,proto3" json:"containers,omitempty"` } -func (m *Containers) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Containers.Unmarshal(m, b) -} -func (m *Containers) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Containers.Marshal(b, m, deterministic) -} -func (m *Containers) XXX_Merge(src proto.Message) { - xxx_messageInfo_Containers.Merge(m, src) +func (x *Containers) Reset() { + *x = Containers{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Containers) XXX_Size() int { - return xxx_messageInfo_Containers.Size(m) + +func (x *Containers) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Containers) XXX_DiscardUnknown() { - xxx_messageInfo_Containers.DiscardUnknown(m) + +func (*Containers) ProtoMessage() {} + +func (x *Containers) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[23] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_Containers proto.InternalMessageInfo +// Deprecated: Use Containers.ProtoReflect.Descriptor instead. +func (*Containers) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{23} +} -func (m *Containers) GetContainers() []*Container { - if m != nil { - return m.Containers +func (x *Containers) GetContainers() []*Container { + if x != nil { + return x.Containers } return nil } type ContainerID struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ContainerID) Reset() { *m = ContainerID{} } -func (m *ContainerID) String() string { return proto.CompactTextString(m) } -func (*ContainerID) ProtoMessage() {} -func (*ContainerID) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{24} + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` } -func (m *ContainerID) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ContainerID.Unmarshal(m, b) -} -func (m *ContainerID) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ContainerID.Marshal(b, m, deterministic) -} -func (m *ContainerID) XXX_Merge(src proto.Message) { - xxx_messageInfo_ContainerID.Merge(m, src) +func (x *ContainerID) Reset() { + *x = ContainerID{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ContainerID) XXX_Size() int { - return xxx_messageInfo_ContainerID.Size(m) + +func (x *ContainerID) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ContainerID) XXX_DiscardUnknown() { - xxx_messageInfo_ContainerID.DiscardUnknown(m) + +func (*ContainerID) ProtoMessage() {} + +func (x *ContainerID) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[24] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ContainerID proto.InternalMessageInfo +// Deprecated: Use ContainerID.ProtoReflect.Descriptor instead. +func (*ContainerID) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{24} +} -func (m *ContainerID) GetId() string { - if m != nil { - return m.Id +func (x *ContainerID) GetId() string { + if x != nil { + return x.Id } return "" } type ContainerIDs struct { - Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ContainerIDs) Reset() { *m = ContainerIDs{} } -func (m *ContainerIDs) String() string { return proto.CompactTextString(m) } -func (*ContainerIDs) ProtoMessage() {} -func (*ContainerIDs) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{25} + Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` } -func (m *ContainerIDs) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ContainerIDs.Unmarshal(m, b) -} -func (m *ContainerIDs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ContainerIDs.Marshal(b, m, deterministic) -} -func (m *ContainerIDs) XXX_Merge(src proto.Message) { - xxx_messageInfo_ContainerIDs.Merge(m, src) +func (x *ContainerIDs) Reset() { + *x = ContainerIDs{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ContainerIDs) XXX_Size() int { - return xxx_messageInfo_ContainerIDs.Size(m) + +func (x *ContainerIDs) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ContainerIDs) XXX_DiscardUnknown() { - xxx_messageInfo_ContainerIDs.DiscardUnknown(m) + +func (*ContainerIDs) ProtoMessage() {} + +func (x *ContainerIDs) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[25] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ContainerIDs proto.InternalMessageInfo +// Deprecated: Use ContainerIDs.ProtoReflect.Descriptor instead. +func (*ContainerIDs) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{25} +} -func (m *ContainerIDs) GetIds() []string { - if m != nil { - return m.Ids +func (x *ContainerIDs) GetIds() []string { + if x != nil { + return x.Ids } return nil } type RemoveContainerOptions struct { - Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` - Force bool `protobuf:"varint,2,opt,name=force,proto3" json:"force,omitempty"` - Step int32 `protobuf:"varint,3,opt,name=step,proto3" json:"step,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *RemoveContainerOptions) Reset() { *m = RemoveContainerOptions{} } -func (m *RemoveContainerOptions) String() string { return proto.CompactTextString(m) } -func (*RemoveContainerOptions) ProtoMessage() {} -func (*RemoveContainerOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{26} + Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` + Force bool `protobuf:"varint,2,opt,name=force,proto3" json:"force,omitempty"` + Step int32 `protobuf:"varint,3,opt,name=step,proto3" json:"step,omitempty"` } -func (m *RemoveContainerOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RemoveContainerOptions.Unmarshal(m, b) -} -func (m *RemoveContainerOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RemoveContainerOptions.Marshal(b, m, deterministic) -} -func (m *RemoveContainerOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_RemoveContainerOptions.Merge(m, src) +func (x *RemoveContainerOptions) Reset() { + *x = RemoveContainerOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *RemoveContainerOptions) XXX_Size() int { - return xxx_messageInfo_RemoveContainerOptions.Size(m) + +func (x *RemoveContainerOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *RemoveContainerOptions) XXX_DiscardUnknown() { - xxx_messageInfo_RemoveContainerOptions.DiscardUnknown(m) + +func (*RemoveContainerOptions) ProtoMessage() {} + +func (x *RemoveContainerOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[26] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_RemoveContainerOptions proto.InternalMessageInfo +// Deprecated: Use RemoveContainerOptions.ProtoReflect.Descriptor instead. +func (*RemoveContainerOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{26} +} -func (m *RemoveContainerOptions) GetIds() []string { - if m != nil { - return m.Ids +func (x *RemoveContainerOptions) GetIds() []string { + if x != nil { + return x.Ids } return nil } -func (m *RemoveContainerOptions) GetForce() bool { - if m != nil { - return m.Force +func (x *RemoveContainerOptions) GetForce() bool { + if x != nil { + return x.Force } return false } -func (m *RemoveContainerOptions) GetStep() int32 { - if m != nil { - return m.Step +func (x *RemoveContainerOptions) GetStep() int32 { + if x != nil { + return x.Step } return 0 } type DissociateContainerOptions struct { - Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *DissociateContainerOptions) Reset() { *m = DissociateContainerOptions{} } -func (m *DissociateContainerOptions) String() string { return proto.CompactTextString(m) } -func (*DissociateContainerOptions) ProtoMessage() {} -func (*DissociateContainerOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{27} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *DissociateContainerOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_DissociateContainerOptions.Unmarshal(m, b) -} -func (m *DissociateContainerOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_DissociateContainerOptions.Marshal(b, m, deterministic) -} -func (m *DissociateContainerOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_DissociateContainerOptions.Merge(m, src) -} -func (m *DissociateContainerOptions) XXX_Size() int { - return xxx_messageInfo_DissociateContainerOptions.Size(m) -} -func (m *DissociateContainerOptions) XXX_DiscardUnknown() { - xxx_messageInfo_DissociateContainerOptions.DiscardUnknown(m) + Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` } -var xxx_messageInfo_DissociateContainerOptions proto.InternalMessageInfo - -func (m *DissociateContainerOptions) GetIds() []string { - if m != nil { - return m.Ids +func (x *DissociateContainerOptions) Reset() { + *x = DissociateContainerOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } - return nil -} - -type ReallocOptions struct { - Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` - Cpu float64 `protobuf:"fixed64,2,opt,name=cpu,proto3" json:"cpu,omitempty"` - Memory int64 `protobuf:"varint,3,opt,name=memory,proto3" json:"memory,omitempty"` - Volumes []string `protobuf:"bytes,4,rep,name=volumes,proto3" json:"volumes,omitempty"` - BindCpuOpt TriOpt `protobuf:"varint,5,opt,name=bind_cpu_opt,json=bindCpuOpt,proto3,enum=pb.TriOpt" json:"bind_cpu_opt,omitempty"` - MemoryLimitOpt TriOpt `protobuf:"varint,6,opt,name=memory_limit_opt,json=memoryLimitOpt,proto3,enum=pb.TriOpt" json:"memory_limit_opt,omitempty"` - CpuRequest float64 `protobuf:"fixed64,7,opt,name=cpu_request,json=cpuRequest,proto3" json:"cpu_request,omitempty"` - CpuLimit float64 `protobuf:"fixed64,8,opt,name=cpu_limit,json=cpuLimit,proto3" json:"cpu_limit,omitempty"` - MemoryRequest int64 `protobuf:"varint,9,opt,name=memory_request,json=memoryRequest,proto3" json:"memory_request,omitempty"` - MemoryLimit int64 `protobuf:"varint,10,opt,name=memory_limit,json=memoryLimit,proto3" json:"memory_limit,omitempty"` - StorageRequest int64 `protobuf:"varint,11,opt,name=storage_request,json=storageRequest,proto3" json:"storage_request,omitempty"` - StorageLimit int64 `protobuf:"varint,12,opt,name=storage_limit,json=storageLimit,proto3" json:"storage_limit,omitempty"` - VolumeRequest []string `protobuf:"bytes,13,rep,name=volume_request,json=volumeRequest,proto3" json:"volume_request,omitempty"` - VolumeLimit []string `protobuf:"bytes,14,rep,name=volume_limit,json=volumeLimit,proto3" json:"volume_limit,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ReallocOptions) Reset() { *m = ReallocOptions{} } -func (m *ReallocOptions) String() string { return proto.CompactTextString(m) } -func (*ReallocOptions) ProtoMessage() {} -func (*ReallocOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{28} } -func (m *ReallocOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ReallocOptions.Unmarshal(m, b) -} -func (m *ReallocOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ReallocOptions.Marshal(b, m, deterministic) -} -func (m *ReallocOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_ReallocOptions.Merge(m, src) -} -func (m *ReallocOptions) XXX_Size() int { - return xxx_messageInfo_ReallocOptions.Size(m) -} -func (m *ReallocOptions) XXX_DiscardUnknown() { - xxx_messageInfo_ReallocOptions.DiscardUnknown(m) +func (x *DissociateContainerOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -var xxx_messageInfo_ReallocOptions proto.InternalMessageInfo - -func (m *ReallocOptions) GetIds() []string { - if m != nil { - return m.Ids - } - return nil -} +func (*DissociateContainerOptions) ProtoMessage() {} -func (m *ReallocOptions) GetCpu() float64 { - if m != nil { - return m.Cpu +func (x *DissociateContainerOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[27] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } - return 0 + return mi.MessageOf(x) } -func (m *ReallocOptions) GetMemory() int64 { - if m != nil { - return m.Memory - } - return 0 +// Deprecated: Use DissociateContainerOptions.ProtoReflect.Descriptor instead. +func (*DissociateContainerOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{27} } -func (m *ReallocOptions) GetVolumes() []string { - if m != nil { - return m.Volumes +func (x *DissociateContainerOptions) GetIds() []string { + if x != nil { + return x.Ids } return nil } -func (m *ReallocOptions) GetBindCpuOpt() TriOpt { - if m != nil { - return m.BindCpuOpt - } - return TriOpt_KEEP -} +type ReallocOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ReallocOptions) GetMemoryLimitOpt() TriOpt { - if m != nil { - return m.MemoryLimitOpt - } - return TriOpt_KEEP + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + BindCpuOpt TriOpt `protobuf:"varint,2,opt,name=bind_cpu_opt,json=bindCpuOpt,proto3,enum=pb.TriOpt" json:"bind_cpu_opt,omitempty"` + ResourceOpts *ResourceOptions `protobuf:"bytes,3,opt,name=resource_opts,json=resourceOpts,proto3" json:"resource_opts,omitempty"` } -func (m *ReallocOptions) GetCpuRequest() float64 { - if m != nil { - return m.CpuRequest +func (x *ReallocOptions) Reset() { + *x = ReallocOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } - return 0 } -func (m *ReallocOptions) GetCpuLimit() float64 { - if m != nil { - return m.CpuLimit - } - return 0 +func (x *ReallocOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ReallocOptions) GetMemoryRequest() int64 { - if m != nil { - return m.MemoryRequest - } - return 0 -} +func (*ReallocOptions) ProtoMessage() {} -func (m *ReallocOptions) GetMemoryLimit() int64 { - if m != nil { - return m.MemoryLimit +func (x *ReallocOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[28] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } - return 0 + return mi.MessageOf(x) } -func (m *ReallocOptions) GetStorageRequest() int64 { - if m != nil { - return m.StorageRequest - } - return 0 +// Deprecated: Use ReallocOptions.ProtoReflect.Descriptor instead. +func (*ReallocOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{28} } -func (m *ReallocOptions) GetStorageLimit() int64 { - if m != nil { - return m.StorageLimit +func (x *ReallocOptions) GetId() string { + if x != nil { + return x.Id } - return 0 + return "" } -func (m *ReallocOptions) GetVolumeRequest() []string { - if m != nil { - return m.VolumeRequest +func (x *ReallocOptions) GetBindCpuOpt() TriOpt { + if x != nil { + return x.BindCpuOpt } - return nil + return TriOpt_KEEP } -func (m *ReallocOptions) GetVolumeLimit() []string { - if m != nil { - return m.VolumeLimit +func (x *ReallocOptions) GetResourceOpts() *ResourceOptions { + if x != nil { + return x.ResourceOpts } return nil } type AddPodOptions struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Desc string `protobuf:"bytes,2,opt,name=desc,proto3" json:"desc,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *AddPodOptions) Reset() { *m = AddPodOptions{} } -func (m *AddPodOptions) String() string { return proto.CompactTextString(m) } -func (*AddPodOptions) ProtoMessage() {} -func (*AddPodOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{29} + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Desc string `protobuf:"bytes,2,opt,name=desc,proto3" json:"desc,omitempty"` } -func (m *AddPodOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_AddPodOptions.Unmarshal(m, b) -} -func (m *AddPodOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_AddPodOptions.Marshal(b, m, deterministic) -} -func (m *AddPodOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_AddPodOptions.Merge(m, src) +func (x *AddPodOptions) Reset() { + *x = AddPodOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *AddPodOptions) XXX_Size() int { - return xxx_messageInfo_AddPodOptions.Size(m) + +func (x *AddPodOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *AddPodOptions) XXX_DiscardUnknown() { - xxx_messageInfo_AddPodOptions.DiscardUnknown(m) + +func (*AddPodOptions) ProtoMessage() {} + +func (x *AddPodOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[29] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_AddPodOptions proto.InternalMessageInfo +// Deprecated: Use AddPodOptions.ProtoReflect.Descriptor instead. +func (*AddPodOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{29} +} -func (m *AddPodOptions) GetName() string { - if m != nil { - return m.Name +func (x *AddPodOptions) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *AddPodOptions) GetDesc() string { - if m != nil { - return m.Desc +func (x *AddPodOptions) GetDesc() string { + if x != nil { + return x.Desc } return "" } type RemovePodOptions struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *RemovePodOptions) Reset() { *m = RemovePodOptions{} } -func (m *RemovePodOptions) String() string { return proto.CompactTextString(m) } -func (*RemovePodOptions) ProtoMessage() {} -func (*RemovePodOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{30} + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` } -func (m *RemovePodOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RemovePodOptions.Unmarshal(m, b) -} -func (m *RemovePodOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RemovePodOptions.Marshal(b, m, deterministic) -} -func (m *RemovePodOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_RemovePodOptions.Merge(m, src) +func (x *RemovePodOptions) Reset() { + *x = RemovePodOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *RemovePodOptions) XXX_Size() int { - return xxx_messageInfo_RemovePodOptions.Size(m) + +func (x *RemovePodOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *RemovePodOptions) XXX_DiscardUnknown() { - xxx_messageInfo_RemovePodOptions.DiscardUnknown(m) + +func (*RemovePodOptions) ProtoMessage() {} + +func (x *RemovePodOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[30] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_RemovePodOptions proto.InternalMessageInfo +// Deprecated: Use RemovePodOptions.ProtoReflect.Descriptor instead. +func (*RemovePodOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{30} +} -func (m *RemovePodOptions) GetName() string { - if m != nil { - return m.Name +func (x *RemovePodOptions) GetName() string { + if x != nil { + return x.Name } return "" } type GetPodOptions struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *GetPodOptions) Reset() { *m = GetPodOptions{} } -func (m *GetPodOptions) String() string { return proto.CompactTextString(m) } -func (*GetPodOptions) ProtoMessage() {} -func (*GetPodOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{31} + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` } -func (m *GetPodOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetPodOptions.Unmarshal(m, b) -} -func (m *GetPodOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetPodOptions.Marshal(b, m, deterministic) -} -func (m *GetPodOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetPodOptions.Merge(m, src) +func (x *GetPodOptions) Reset() { + *x = GetPodOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *GetPodOptions) XXX_Size() int { - return xxx_messageInfo_GetPodOptions.Size(m) + +func (x *GetPodOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *GetPodOptions) XXX_DiscardUnknown() { - xxx_messageInfo_GetPodOptions.DiscardUnknown(m) + +func (*GetPodOptions) ProtoMessage() {} + +func (x *GetPodOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[31] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_GetPodOptions proto.InternalMessageInfo +// Deprecated: Use GetPodOptions.ProtoReflect.Descriptor instead. +func (*GetPodOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{31} +} -func (m *GetPodOptions) GetName() string { - if m != nil { - return m.Name +func (x *GetPodOptions) GetName() string { + if x != nil { + return x.Name } return "" } type AddNodeOptions struct { - Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` - Endpoint string `protobuf:"bytes,2,opt,name=endpoint,proto3" json:"endpoint,omitempty"` - Podname string `protobuf:"bytes,3,opt,name=podname,proto3" json:"podname,omitempty"` - Ca string `protobuf:"bytes,4,opt,name=ca,proto3" json:"ca,omitempty"` - Cert string `protobuf:"bytes,5,opt,name=cert,proto3" json:"cert,omitempty"` - Key string `protobuf:"bytes,6,opt,name=key,proto3" json:"key,omitempty"` - Cpu int32 `protobuf:"varint,7,opt,name=cpu,proto3" json:"cpu,omitempty"` - Share int32 `protobuf:"varint,8,opt,name=share,proto3" json:"share,omitempty"` - Memory int64 `protobuf:"varint,9,opt,name=memory,proto3" json:"memory,omitempty"` - Labels map[string]string `protobuf:"bytes,10,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Numa map[string]string `protobuf:"bytes,11,rep,name=numa,proto3" json:"numa,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - NumaMemory map[string]int64 `protobuf:"bytes,12,rep,name=numa_memory,json=numaMemory,proto3" json:"numa_memory,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Storage int64 `protobuf:"varint,13,opt,name=storage,proto3" json:"storage,omitempty"` - VolumeMap map[string]int64 `protobuf:"bytes,14,rep,name=volume_map,json=volumeMap,proto3" json:"volume_map,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *AddNodeOptions) Reset() { *m = AddNodeOptions{} } -func (m *AddNodeOptions) String() string { return proto.CompactTextString(m) } -func (*AddNodeOptions) ProtoMessage() {} -func (*AddNodeOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{32} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *AddNodeOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_AddNodeOptions.Unmarshal(m, b) -} -func (m *AddNodeOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_AddNodeOptions.Marshal(b, m, deterministic) + Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` + Endpoint string `protobuf:"bytes,2,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + Podname string `protobuf:"bytes,3,opt,name=podname,proto3" json:"podname,omitempty"` + Ca string `protobuf:"bytes,4,opt,name=ca,proto3" json:"ca,omitempty"` + Cert string `protobuf:"bytes,5,opt,name=cert,proto3" json:"cert,omitempty"` + Key string `protobuf:"bytes,6,opt,name=key,proto3" json:"key,omitempty"` + Cpu int32 `protobuf:"varint,7,opt,name=cpu,proto3" json:"cpu,omitempty"` + Share int32 `protobuf:"varint,8,opt,name=share,proto3" json:"share,omitempty"` + Memory int64 `protobuf:"varint,9,opt,name=memory,proto3" json:"memory,omitempty"` + Labels map[string]string `protobuf:"bytes,10,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Numa map[string]string `protobuf:"bytes,11,rep,name=numa,proto3" json:"numa,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + NumaMemory map[string]int64 `protobuf:"bytes,12,rep,name=numa_memory,json=numaMemory,proto3" json:"numa_memory,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Storage int64 `protobuf:"varint,13,opt,name=storage,proto3" json:"storage,omitempty"` + VolumeMap map[string]int64 `protobuf:"bytes,14,rep,name=volume_map,json=volumeMap,proto3" json:"volume_map,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` } -func (m *AddNodeOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_AddNodeOptions.Merge(m, src) + +func (x *AddNodeOptions) Reset() { + *x = AddNodeOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *AddNodeOptions) XXX_Size() int { - return xxx_messageInfo_AddNodeOptions.Size(m) + +func (x *AddNodeOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *AddNodeOptions) XXX_DiscardUnknown() { - xxx_messageInfo_AddNodeOptions.DiscardUnknown(m) + +func (*AddNodeOptions) ProtoMessage() {} + +func (x *AddNodeOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_AddNodeOptions proto.InternalMessageInfo +// Deprecated: Use AddNodeOptions.ProtoReflect.Descriptor instead. +func (*AddNodeOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{32} +} -func (m *AddNodeOptions) GetNodename() string { - if m != nil { - return m.Nodename +func (x *AddNodeOptions) GetNodename() string { + if x != nil { + return x.Nodename } return "" } -func (m *AddNodeOptions) GetEndpoint() string { - if m != nil { - return m.Endpoint +func (x *AddNodeOptions) GetEndpoint() string { + if x != nil { + return x.Endpoint } return "" } -func (m *AddNodeOptions) GetPodname() string { - if m != nil { - return m.Podname +func (x *AddNodeOptions) GetPodname() string { + if x != nil { + return x.Podname } return "" } -func (m *AddNodeOptions) GetCa() string { - if m != nil { - return m.Ca +func (x *AddNodeOptions) GetCa() string { + if x != nil { + return x.Ca } return "" } -func (m *AddNodeOptions) GetCert() string { - if m != nil { - return m.Cert +func (x *AddNodeOptions) GetCert() string { + if x != nil { + return x.Cert } return "" } -func (m *AddNodeOptions) GetKey() string { - if m != nil { - return m.Key +func (x *AddNodeOptions) GetKey() string { + if x != nil { + return x.Key } return "" } -func (m *AddNodeOptions) GetCpu() int32 { - if m != nil { - return m.Cpu +func (x *AddNodeOptions) GetCpu() int32 { + if x != nil { + return x.Cpu } return 0 } -func (m *AddNodeOptions) GetShare() int32 { - if m != nil { - return m.Share +func (x *AddNodeOptions) GetShare() int32 { + if x != nil { + return x.Share } return 0 } -func (m *AddNodeOptions) GetMemory() int64 { - if m != nil { - return m.Memory +func (x *AddNodeOptions) GetMemory() int64 { + if x != nil { + return x.Memory } return 0 } -func (m *AddNodeOptions) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *AddNodeOptions) GetLabels() map[string]string { + if x != nil { + return x.Labels } return nil } -func (m *AddNodeOptions) GetNuma() map[string]string { - if m != nil { - return m.Numa +func (x *AddNodeOptions) GetNuma() map[string]string { + if x != nil { + return x.Numa } return nil } -func (m *AddNodeOptions) GetNumaMemory() map[string]int64 { - if m != nil { - return m.NumaMemory +func (x *AddNodeOptions) GetNumaMemory() map[string]int64 { + if x != nil { + return x.NumaMemory } return nil } -func (m *AddNodeOptions) GetStorage() int64 { - if m != nil { - return m.Storage +func (x *AddNodeOptions) GetStorage() int64 { + if x != nil { + return x.Storage } return 0 } -func (m *AddNodeOptions) GetVolumeMap() map[string]int64 { - if m != nil { - return m.VolumeMap +func (x *AddNodeOptions) GetVolumeMap() map[string]int64 { + if x != nil { + return x.VolumeMap } return nil } type RemoveNodeOptions struct { - Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *RemoveNodeOptions) Reset() { *m = RemoveNodeOptions{} } -func (m *RemoveNodeOptions) String() string { return proto.CompactTextString(m) } -func (*RemoveNodeOptions) ProtoMessage() {} -func (*RemoveNodeOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{33} + Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` } -func (m *RemoveNodeOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RemoveNodeOptions.Unmarshal(m, b) -} -func (m *RemoveNodeOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RemoveNodeOptions.Marshal(b, m, deterministic) -} -func (m *RemoveNodeOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_RemoveNodeOptions.Merge(m, src) +func (x *RemoveNodeOptions) Reset() { + *x = RemoveNodeOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *RemoveNodeOptions) XXX_Size() int { - return xxx_messageInfo_RemoveNodeOptions.Size(m) + +func (x *RemoveNodeOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *RemoveNodeOptions) XXX_DiscardUnknown() { - xxx_messageInfo_RemoveNodeOptions.DiscardUnknown(m) + +func (*RemoveNodeOptions) ProtoMessage() {} + +func (x *RemoveNodeOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[33] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_RemoveNodeOptions proto.InternalMessageInfo +// Deprecated: Use RemoveNodeOptions.ProtoReflect.Descriptor instead. +func (*RemoveNodeOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{33} +} -func (m *RemoveNodeOptions) GetNodename() string { - if m != nil { - return m.Nodename +func (x *RemoveNodeOptions) GetNodename() string { + if x != nil { + return x.Nodename } return "" } type GetNodeOptions struct { - Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` - Labels map[string]string `protobuf:"bytes,2,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *GetNodeOptions) Reset() { *m = GetNodeOptions{} } -func (m *GetNodeOptions) String() string { return proto.CompactTextString(m) } -func (*GetNodeOptions) ProtoMessage() {} -func (*GetNodeOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{34} + Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,omitempty"` + Labels map[string]string `protobuf:"bytes,2,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (m *GetNodeOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetNodeOptions.Unmarshal(m, b) -} -func (m *GetNodeOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetNodeOptions.Marshal(b, m, deterministic) -} -func (m *GetNodeOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetNodeOptions.Merge(m, src) +func (x *GetNodeOptions) Reset() { + *x = GetNodeOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *GetNodeOptions) XXX_Size() int { - return xxx_messageInfo_GetNodeOptions.Size(m) + +func (x *GetNodeOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *GetNodeOptions) XXX_DiscardUnknown() { - xxx_messageInfo_GetNodeOptions.DiscardUnknown(m) + +func (*GetNodeOptions) ProtoMessage() {} + +func (x *GetNodeOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[34] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_GetNodeOptions proto.InternalMessageInfo +// Deprecated: Use GetNodeOptions.ProtoReflect.Descriptor instead. +func (*GetNodeOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{34} +} -func (m *GetNodeOptions) GetNodename() string { - if m != nil { - return m.Nodename +func (x *GetNodeOptions) GetNodename() string { + if x != nil { + return x.Nodename } return "" } -func (m *GetNodeOptions) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *GetNodeOptions) GetLabels() map[string]string { + if x != nil { + return x.Labels } return nil } type GetNodeResourceOptions struct { - Opts *GetNodeOptions `protobuf:"bytes,1,opt,name=opts,proto3" json:"opts,omitempty"` - Fix bool `protobuf:"varint,2,opt,name=fix,proto3" json:"fix,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *GetNodeResourceOptions) Reset() { *m = GetNodeResourceOptions{} } -func (m *GetNodeResourceOptions) String() string { return proto.CompactTextString(m) } -func (*GetNodeResourceOptions) ProtoMessage() {} -func (*GetNodeResourceOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{35} + Opts *GetNodeOptions `protobuf:"bytes,1,opt,name=opts,proto3" json:"opts,omitempty"` + Fix bool `protobuf:"varint,2,opt,name=fix,proto3" json:"fix,omitempty"` } -func (m *GetNodeResourceOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetNodeResourceOptions.Unmarshal(m, b) -} -func (m *GetNodeResourceOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetNodeResourceOptions.Marshal(b, m, deterministic) -} -func (m *GetNodeResourceOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetNodeResourceOptions.Merge(m, src) +func (x *GetNodeResourceOptions) Reset() { + *x = GetNodeResourceOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *GetNodeResourceOptions) XXX_Size() int { - return xxx_messageInfo_GetNodeResourceOptions.Size(m) + +func (x *GetNodeResourceOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *GetNodeResourceOptions) XXX_DiscardUnknown() { - xxx_messageInfo_GetNodeResourceOptions.DiscardUnknown(m) + +func (*GetNodeResourceOptions) ProtoMessage() {} + +func (x *GetNodeResourceOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[35] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_GetNodeResourceOptions proto.InternalMessageInfo +// Deprecated: Use GetNodeResourceOptions.ProtoReflect.Descriptor instead. +func (*GetNodeResourceOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{35} +} -func (m *GetNodeResourceOptions) GetOpts() *GetNodeOptions { - if m != nil { - return m.Opts +func (x *GetNodeResourceOptions) GetOpts() *GetNodeOptions { + if x != nil { + return x.Opts } return nil } -func (m *GetNodeResourceOptions) GetFix() bool { - if m != nil { - return m.Fix +func (x *GetNodeResourceOptions) GetFix() bool { + if x != nil { + return x.Fix } return false } type ListNodesOptions struct { - Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` - All bool `protobuf:"varint,2,opt,name=all,proto3" json:"all,omitempty"` - Labels map[string]string `protobuf:"bytes,3,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ListNodesOptions) Reset() { *m = ListNodesOptions{} } -func (m *ListNodesOptions) String() string { return proto.CompactTextString(m) } -func (*ListNodesOptions) ProtoMessage() {} -func (*ListNodesOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{36} + Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` + All bool `protobuf:"varint,2,opt,name=all,proto3" json:"all,omitempty"` + Labels map[string]string `protobuf:"bytes,3,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (m *ListNodesOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ListNodesOptions.Unmarshal(m, b) -} -func (m *ListNodesOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ListNodesOptions.Marshal(b, m, deterministic) -} -func (m *ListNodesOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_ListNodesOptions.Merge(m, src) +func (x *ListNodesOptions) Reset() { + *x = ListNodesOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ListNodesOptions) XXX_Size() int { - return xxx_messageInfo_ListNodesOptions.Size(m) + +func (x *ListNodesOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ListNodesOptions) XXX_DiscardUnknown() { - xxx_messageInfo_ListNodesOptions.DiscardUnknown(m) + +func (*ListNodesOptions) ProtoMessage() {} + +func (x *ListNodesOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[36] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ListNodesOptions proto.InternalMessageInfo +// Deprecated: Use ListNodesOptions.ProtoReflect.Descriptor instead. +func (*ListNodesOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{36} +} -func (m *ListNodesOptions) GetPodname() string { - if m != nil { - return m.Podname +func (x *ListNodesOptions) GetPodname() string { + if x != nil { + return x.Podname } return "" } -func (m *ListNodesOptions) GetAll() bool { - if m != nil { - return m.All +func (x *ListNodesOptions) GetAll() bool { + if x != nil { + return x.All } return false } -func (m *ListNodesOptions) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *ListNodesOptions) GetLabels() map[string]string { + if x != nil { + return x.Labels } return nil } type Build struct { - Base string `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - Repo string `protobuf:"bytes,2,opt,name=repo,proto3" json:"repo,omitempty"` - Version string `protobuf:"bytes,3,opt,name=version,proto3" json:"version,omitempty"` - Dir string `protobuf:"bytes,4,opt,name=dir,proto3" json:"dir,omitempty"` - Submodule bool `protobuf:"varint,5,opt,name=submodule,proto3" json:"submodule,omitempty"` - Commands []string `protobuf:"bytes,6,rep,name=commands,proto3" json:"commands,omitempty"` - Envs map[string]string `protobuf:"bytes,7,rep,name=envs,proto3" json:"envs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Args map[string]string `protobuf:"bytes,8,rep,name=args,proto3" json:"args,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Labels map[string]string `protobuf:"bytes,9,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Artifacts map[string]string `protobuf:"bytes,10,rep,name=artifacts,proto3" json:"artifacts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Cache map[string]string `protobuf:"bytes,11,rep,name=cache,proto3" json:"cache,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - StopSignal string `protobuf:"bytes,12,opt,name=stop_signal,json=stopSignal,proto3" json:"stop_signal,omitempty"` - Security bool `protobuf:"varint,13,opt,name=security,proto3" json:"security,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *Build) Reset() { *m = Build{} } -func (m *Build) String() string { return proto.CompactTextString(m) } -func (*Build) ProtoMessage() {} -func (*Build) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{37} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Build) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Build.Unmarshal(m, b) + Base string `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Repo string `protobuf:"bytes,2,opt,name=repo,proto3" json:"repo,omitempty"` + Version string `protobuf:"bytes,3,opt,name=version,proto3" json:"version,omitempty"` + Dir string `protobuf:"bytes,4,opt,name=dir,proto3" json:"dir,omitempty"` + Submodule bool `protobuf:"varint,5,opt,name=submodule,proto3" json:"submodule,omitempty"` + Commands []string `protobuf:"bytes,6,rep,name=commands,proto3" json:"commands,omitempty"` + Envs map[string]string `protobuf:"bytes,7,rep,name=envs,proto3" json:"envs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Args map[string]string `protobuf:"bytes,8,rep,name=args,proto3" json:"args,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Labels map[string]string `protobuf:"bytes,9,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Artifacts map[string]string `protobuf:"bytes,10,rep,name=artifacts,proto3" json:"artifacts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Cache map[string]string `protobuf:"bytes,11,rep,name=cache,proto3" json:"cache,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + StopSignal string `protobuf:"bytes,12,opt,name=stop_signal,json=stopSignal,proto3" json:"stop_signal,omitempty"` + Security bool `protobuf:"varint,13,opt,name=security,proto3" json:"security,omitempty"` } -func (m *Build) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Build.Marshal(b, m, deterministic) -} -func (m *Build) XXX_Merge(src proto.Message) { - xxx_messageInfo_Build.Merge(m, src) + +func (x *Build) Reset() { + *x = Build{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Build) XXX_Size() int { - return xxx_messageInfo_Build.Size(m) + +func (x *Build) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Build) XXX_DiscardUnknown() { - xxx_messageInfo_Build.DiscardUnknown(m) + +func (*Build) ProtoMessage() {} + +func (x *Build) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[37] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_Build proto.InternalMessageInfo +// Deprecated: Use Build.ProtoReflect.Descriptor instead. +func (*Build) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{37} +} -func (m *Build) GetBase() string { - if m != nil { - return m.Base +func (x *Build) GetBase() string { + if x != nil { + return x.Base } return "" } -func (m *Build) GetRepo() string { - if m != nil { - return m.Repo +func (x *Build) GetRepo() string { + if x != nil { + return x.Repo } return "" } -func (m *Build) GetVersion() string { - if m != nil { - return m.Version +func (x *Build) GetVersion() string { + if x != nil { + return x.Version } return "" } -func (m *Build) GetDir() string { - if m != nil { - return m.Dir +func (x *Build) GetDir() string { + if x != nil { + return x.Dir } return "" } -func (m *Build) GetSubmodule() bool { - if m != nil { - return m.Submodule +func (x *Build) GetSubmodule() bool { + if x != nil { + return x.Submodule } return false } -func (m *Build) GetCommands() []string { - if m != nil { - return m.Commands +func (x *Build) GetCommands() []string { + if x != nil { + return x.Commands } return nil } -func (m *Build) GetEnvs() map[string]string { - if m != nil { - return m.Envs +func (x *Build) GetEnvs() map[string]string { + if x != nil { + return x.Envs } return nil } -func (m *Build) GetArgs() map[string]string { - if m != nil { - return m.Args +func (x *Build) GetArgs() map[string]string { + if x != nil { + return x.Args } return nil } -func (m *Build) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *Build) GetLabels() map[string]string { + if x != nil { + return x.Labels } return nil } -func (m *Build) GetArtifacts() map[string]string { - if m != nil { - return m.Artifacts +func (x *Build) GetArtifacts() map[string]string { + if x != nil { + return x.Artifacts } return nil } -func (m *Build) GetCache() map[string]string { - if m != nil { - return m.Cache +func (x *Build) GetCache() map[string]string { + if x != nil { + return x.Cache } return nil } -func (m *Build) GetStopSignal() string { - if m != nil { - return m.StopSignal +func (x *Build) GetStopSignal() string { + if x != nil { + return x.StopSignal } return "" } -func (m *Build) GetSecurity() bool { - if m != nil { - return m.Security +func (x *Build) GetSecurity() bool { + if x != nil { + return x.Security } return false } type Builds struct { - Stages []string `protobuf:"bytes,1,rep,name=stages,proto3" json:"stages,omitempty"` - Builds map[string]*Build `protobuf:"bytes,2,rep,name=builds,proto3" json:"builds,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Builds) Reset() { *m = Builds{} } -func (m *Builds) String() string { return proto.CompactTextString(m) } -func (*Builds) ProtoMessage() {} -func (*Builds) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{38} + Stages []string `protobuf:"bytes,1,rep,name=stages,proto3" json:"stages,omitempty"` + Builds map[string]*Build `protobuf:"bytes,2,rep,name=builds,proto3" json:"builds,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (m *Builds) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Builds.Unmarshal(m, b) -} -func (m *Builds) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Builds.Marshal(b, m, deterministic) -} -func (m *Builds) XXX_Merge(src proto.Message) { - xxx_messageInfo_Builds.Merge(m, src) +func (x *Builds) Reset() { + *x = Builds{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Builds) XXX_Size() int { - return xxx_messageInfo_Builds.Size(m) + +func (x *Builds) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *Builds) XXX_DiscardUnknown() { - xxx_messageInfo_Builds.DiscardUnknown(m) + +func (*Builds) ProtoMessage() {} + +func (x *Builds) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[38] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_Builds proto.InternalMessageInfo +// Deprecated: Use Builds.ProtoReflect.Descriptor instead. +func (*Builds) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{38} +} -func (m *Builds) GetStages() []string { - if m != nil { - return m.Stages +func (x *Builds) GetStages() []string { + if x != nil { + return x.Stages } return nil } -func (m *Builds) GetBuilds() map[string]*Build { - if m != nil { - return m.Builds +func (x *Builds) GetBuilds() map[string]*Build { + if x != nil { + return x.Builds } return nil } type BuildImageOptions struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - User string `protobuf:"bytes,2,opt,name=user,proto3" json:"user,omitempty"` - Uid int32 `protobuf:"varint,3,opt,name=uid,proto3" json:"uid,omitempty"` - Tags []string `protobuf:"bytes,4,rep,name=tags,proto3" json:"tags,omitempty"` - Builds *Builds `protobuf:"bytes,5,opt,name=builds,proto3" json:"builds,omitempty"` - Tar []byte `protobuf:"bytes,6,opt,name=tar,proto3" json:"tar,omitempty"` - BuildMethod BuildImageOptions_BuildMethod `protobuf:"varint,7,opt,name=build_method,json=buildMethod,proto3,enum=pb.BuildImageOptions_BuildMethod" json:"build_method,omitempty"` - ExistId string `protobuf:"bytes,8,opt,name=exist_id,json=existId,proto3" json:"exist_id,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *BuildImageOptions) Reset() { *m = BuildImageOptions{} } -func (m *BuildImageOptions) String() string { return proto.CompactTextString(m) } -func (*BuildImageOptions) ProtoMessage() {} -func (*BuildImageOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{39} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *BuildImageOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_BuildImageOptions.Unmarshal(m, b) -} -func (m *BuildImageOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_BuildImageOptions.Marshal(b, m, deterministic) + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + User string `protobuf:"bytes,2,opt,name=user,proto3" json:"user,omitempty"` + Uid int32 `protobuf:"varint,3,opt,name=uid,proto3" json:"uid,omitempty"` + Tags []string `protobuf:"bytes,4,rep,name=tags,proto3" json:"tags,omitempty"` + Builds *Builds `protobuf:"bytes,5,opt,name=builds,proto3" json:"builds,omitempty"` + Tar []byte `protobuf:"bytes,6,opt,name=tar,proto3" json:"tar,omitempty"` + BuildMethod BuildImageOptions_BuildMethod `protobuf:"varint,7,opt,name=build_method,json=buildMethod,proto3,enum=pb.BuildImageOptions_BuildMethod" json:"build_method,omitempty"` + ExistId string `protobuf:"bytes,8,opt,name=exist_id,json=existId,proto3" json:"exist_id,omitempty"` } -func (m *BuildImageOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_BuildImageOptions.Merge(m, src) + +func (x *BuildImageOptions) Reset() { + *x = BuildImageOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *BuildImageOptions) XXX_Size() int { - return xxx_messageInfo_BuildImageOptions.Size(m) + +func (x *BuildImageOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *BuildImageOptions) XXX_DiscardUnknown() { - xxx_messageInfo_BuildImageOptions.DiscardUnknown(m) + +func (*BuildImageOptions) ProtoMessage() {} + +func (x *BuildImageOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[39] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_BuildImageOptions proto.InternalMessageInfo +// Deprecated: Use BuildImageOptions.ProtoReflect.Descriptor instead. +func (*BuildImageOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{39} +} -func (m *BuildImageOptions) GetName() string { - if m != nil { - return m.Name +func (x *BuildImageOptions) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *BuildImageOptions) GetUser() string { - if m != nil { - return m.User +func (x *BuildImageOptions) GetUser() string { + if x != nil { + return x.User } return "" } -func (m *BuildImageOptions) GetUid() int32 { - if m != nil { - return m.Uid +func (x *BuildImageOptions) GetUid() int32 { + if x != nil { + return x.Uid } return 0 } -func (m *BuildImageOptions) GetTags() []string { - if m != nil { - return m.Tags +func (x *BuildImageOptions) GetTags() []string { + if x != nil { + return x.Tags } return nil } -func (m *BuildImageOptions) GetBuilds() *Builds { - if m != nil { - return m.Builds +func (x *BuildImageOptions) GetBuilds() *Builds { + if x != nil { + return x.Builds } return nil } -func (m *BuildImageOptions) GetTar() []byte { - if m != nil { - return m.Tar +func (x *BuildImageOptions) GetTar() []byte { + if x != nil { + return x.Tar } return nil } -func (m *BuildImageOptions) GetBuildMethod() BuildImageOptions_BuildMethod { - if m != nil { - return m.BuildMethod +func (x *BuildImageOptions) GetBuildMethod() BuildImageOptions_BuildMethod { + if x != nil { + return x.BuildMethod } return BuildImageOptions_SCM } -func (m *BuildImageOptions) GetExistId() string { - if m != nil { - return m.ExistId +func (x *BuildImageOptions) GetExistId() string { + if x != nil { + return x.ExistId } return "" } type HookOptions struct { - AfterStart []string `protobuf:"bytes,1,rep,name=after_start,json=afterStart,proto3" json:"after_start,omitempty"` - BeforeStop []string `protobuf:"bytes,2,rep,name=before_stop,json=beforeStop,proto3" json:"before_stop,omitempty"` - Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *HookOptions) Reset() { *m = HookOptions{} } -func (m *HookOptions) String() string { return proto.CompactTextString(m) } -func (*HookOptions) ProtoMessage() {} -func (*HookOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{40} + AfterStart []string `protobuf:"bytes,1,rep,name=after_start,json=afterStart,proto3" json:"after_start,omitempty"` + BeforeStop []string `protobuf:"bytes,2,rep,name=before_stop,json=beforeStop,proto3" json:"before_stop,omitempty"` + Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"` } -func (m *HookOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_HookOptions.Unmarshal(m, b) -} -func (m *HookOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_HookOptions.Marshal(b, m, deterministic) -} -func (m *HookOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_HookOptions.Merge(m, src) +func (x *HookOptions) Reset() { + *x = HookOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *HookOptions) XXX_Size() int { - return xxx_messageInfo_HookOptions.Size(m) + +func (x *HookOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *HookOptions) XXX_DiscardUnknown() { - xxx_messageInfo_HookOptions.DiscardUnknown(m) + +func (*HookOptions) ProtoMessage() {} + +func (x *HookOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[40] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_HookOptions proto.InternalMessageInfo +// Deprecated: Use HookOptions.ProtoReflect.Descriptor instead. +func (*HookOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{40} +} -func (m *HookOptions) GetAfterStart() []string { - if m != nil { - return m.AfterStart +func (x *HookOptions) GetAfterStart() []string { + if x != nil { + return x.AfterStart } return nil } -func (m *HookOptions) GetBeforeStop() []string { - if m != nil { - return m.BeforeStop +func (x *HookOptions) GetBeforeStop() []string { + if x != nil { + return x.BeforeStop } return nil } -func (m *HookOptions) GetForce() bool { - if m != nil { - return m.Force +func (x *HookOptions) GetForce() bool { + if x != nil { + return x.Force } return false } type HealthCheckOptions struct { - TcpPorts []string `protobuf:"bytes,1,rep,name=tcp_ports,json=tcpPorts,proto3" json:"tcp_ports,omitempty"` - HttpPort string `protobuf:"bytes,2,opt,name=http_port,json=httpPort,proto3" json:"http_port,omitempty"` - Url string `protobuf:"bytes,3,opt,name=url,proto3" json:"url,omitempty"` - Code int32 `protobuf:"varint,4,opt,name=code,proto3" json:"code,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *HealthCheckOptions) Reset() { *m = HealthCheckOptions{} } -func (m *HealthCheckOptions) String() string { return proto.CompactTextString(m) } -func (*HealthCheckOptions) ProtoMessage() {} -func (*HealthCheckOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{41} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *HealthCheckOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_HealthCheckOptions.Unmarshal(m, b) -} -func (m *HealthCheckOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_HealthCheckOptions.Marshal(b, m, deterministic) + TcpPorts []string `protobuf:"bytes,1,rep,name=tcp_ports,json=tcpPorts,proto3" json:"tcp_ports,omitempty"` + HttpPort string `protobuf:"bytes,2,opt,name=http_port,json=httpPort,proto3" json:"http_port,omitempty"` + Url string `protobuf:"bytes,3,opt,name=url,proto3" json:"url,omitempty"` + Code int32 `protobuf:"varint,4,opt,name=code,proto3" json:"code,omitempty"` } -func (m *HealthCheckOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_HealthCheckOptions.Merge(m, src) + +func (x *HealthCheckOptions) Reset() { + *x = HealthCheckOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *HealthCheckOptions) XXX_Size() int { - return xxx_messageInfo_HealthCheckOptions.Size(m) + +func (x *HealthCheckOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *HealthCheckOptions) XXX_DiscardUnknown() { - xxx_messageInfo_HealthCheckOptions.DiscardUnknown(m) + +func (*HealthCheckOptions) ProtoMessage() {} + +func (x *HealthCheckOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[41] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_HealthCheckOptions proto.InternalMessageInfo +// Deprecated: Use HealthCheckOptions.ProtoReflect.Descriptor instead. +func (*HealthCheckOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{41} +} -func (m *HealthCheckOptions) GetTcpPorts() []string { - if m != nil { - return m.TcpPorts +func (x *HealthCheckOptions) GetTcpPorts() []string { + if x != nil { + return x.TcpPorts } return nil } -func (m *HealthCheckOptions) GetHttpPort() string { - if m != nil { - return m.HttpPort +func (x *HealthCheckOptions) GetHttpPort() string { + if x != nil { + return x.HttpPort } return "" } -func (m *HealthCheckOptions) GetUrl() string { - if m != nil { - return m.Url +func (x *HealthCheckOptions) GetUrl() string { + if x != nil { + return x.Url } return "" } -func (m *HealthCheckOptions) GetCode() int32 { - if m != nil { - return m.Code +func (x *HealthCheckOptions) GetCode() int32 { + if x != nil { + return x.Code } return 0 } type LogOptions struct { - Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"` - Config map[string]string `protobuf:"bytes,2,rep,name=config,proto3" json:"config,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *LogOptions) Reset() { *m = LogOptions{} } -func (m *LogOptions) String() string { return proto.CompactTextString(m) } -func (*LogOptions) ProtoMessage() {} -func (*LogOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{42} + Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"` + Config map[string]string `protobuf:"bytes,2,rep,name=config,proto3" json:"config,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (m *LogOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_LogOptions.Unmarshal(m, b) -} -func (m *LogOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_LogOptions.Marshal(b, m, deterministic) -} -func (m *LogOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_LogOptions.Merge(m, src) +func (x *LogOptions) Reset() { + *x = LogOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *LogOptions) XXX_Size() int { - return xxx_messageInfo_LogOptions.Size(m) + +func (x *LogOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *LogOptions) XXX_DiscardUnknown() { - xxx_messageInfo_LogOptions.DiscardUnknown(m) + +func (*LogOptions) ProtoMessage() {} + +func (x *LogOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[42] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_LogOptions proto.InternalMessageInfo +// Deprecated: Use LogOptions.ProtoReflect.Descriptor instead. +func (*LogOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{42} +} -func (m *LogOptions) GetType() string { - if m != nil { - return m.Type +func (x *LogOptions) GetType() string { + if x != nil { + return x.Type } return "" } -func (m *LogOptions) GetConfig() map[string]string { - if m != nil { - return m.Config +func (x *LogOptions) GetConfig() map[string]string { + if x != nil { + return x.Config } return nil } type EntrypointOptions struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Command string `protobuf:"bytes,2,opt,name=command,proto3" json:"command,omitempty"` - Privileged bool `protobuf:"varint,3,opt,name=privileged,proto3" json:"privileged,omitempty"` - Dir string `protobuf:"bytes,4,opt,name=dir,proto3" json:"dir,omitempty"` - Log *LogOptions `protobuf:"bytes,5,opt,name=log,proto3" json:"log,omitempty"` - Publish []string `protobuf:"bytes,6,rep,name=publish,proto3" json:"publish,omitempty"` - Healthcheck *HealthCheckOptions `protobuf:"bytes,7,opt,name=healthcheck,proto3" json:"healthcheck,omitempty"` - Hook *HookOptions `protobuf:"bytes,8,opt,name=hook,proto3" json:"hook,omitempty"` - RestartPolicy string `protobuf:"bytes,9,opt,name=restart_policy,json=restartPolicy,proto3" json:"restart_policy,omitempty"` - Sysctls map[string]string `protobuf:"bytes,10,rep,name=sysctls,proto3" json:"sysctls,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *EntrypointOptions) Reset() { *m = EntrypointOptions{} } -func (m *EntrypointOptions) String() string { return proto.CompactTextString(m) } -func (*EntrypointOptions) ProtoMessage() {} -func (*EntrypointOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{43} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *EntrypointOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_EntrypointOptions.Unmarshal(m, b) -} -func (m *EntrypointOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_EntrypointOptions.Marshal(b, m, deterministic) + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Command string `protobuf:"bytes,2,opt,name=command,proto3" json:"command,omitempty"` + Privileged bool `protobuf:"varint,3,opt,name=privileged,proto3" json:"privileged,omitempty"` + Dir string `protobuf:"bytes,4,opt,name=dir,proto3" json:"dir,omitempty"` + Log *LogOptions `protobuf:"bytes,5,opt,name=log,proto3" json:"log,omitempty"` + Publish []string `protobuf:"bytes,6,rep,name=publish,proto3" json:"publish,omitempty"` + Healthcheck *HealthCheckOptions `protobuf:"bytes,7,opt,name=healthcheck,proto3" json:"healthcheck,omitempty"` + Hook *HookOptions `protobuf:"bytes,8,opt,name=hook,proto3" json:"hook,omitempty"` + RestartPolicy string `protobuf:"bytes,9,opt,name=restart_policy,json=restartPolicy,proto3" json:"restart_policy,omitempty"` + Sysctls map[string]string `protobuf:"bytes,10,rep,name=sysctls,proto3" json:"sysctls,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (m *EntrypointOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_EntrypointOptions.Merge(m, src) + +func (x *EntrypointOptions) Reset() { + *x = EntrypointOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *EntrypointOptions) XXX_Size() int { - return xxx_messageInfo_EntrypointOptions.Size(m) + +func (x *EntrypointOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *EntrypointOptions) XXX_DiscardUnknown() { - xxx_messageInfo_EntrypointOptions.DiscardUnknown(m) + +func (*EntrypointOptions) ProtoMessage() {} + +func (x *EntrypointOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[43] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_EntrypointOptions proto.InternalMessageInfo +// Deprecated: Use EntrypointOptions.ProtoReflect.Descriptor instead. +func (*EntrypointOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{43} +} -func (m *EntrypointOptions) GetName() string { - if m != nil { - return m.Name +func (x *EntrypointOptions) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *EntrypointOptions) GetCommand() string { - if m != nil { - return m.Command +func (x *EntrypointOptions) GetCommand() string { + if x != nil { + return x.Command } return "" } -func (m *EntrypointOptions) GetPrivileged() bool { - if m != nil { - return m.Privileged +func (x *EntrypointOptions) GetPrivileged() bool { + if x != nil { + return x.Privileged } return false } -func (m *EntrypointOptions) GetDir() string { - if m != nil { - return m.Dir +func (x *EntrypointOptions) GetDir() string { + if x != nil { + return x.Dir } return "" } -func (m *EntrypointOptions) GetLog() *LogOptions { - if m != nil { - return m.Log +func (x *EntrypointOptions) GetLog() *LogOptions { + if x != nil { + return x.Log } return nil } -func (m *EntrypointOptions) GetPublish() []string { - if m != nil { - return m.Publish +func (x *EntrypointOptions) GetPublish() []string { + if x != nil { + return x.Publish } return nil } -func (m *EntrypointOptions) GetHealthcheck() *HealthCheckOptions { - if m != nil { - return m.Healthcheck +func (x *EntrypointOptions) GetHealthcheck() *HealthCheckOptions { + if x != nil { + return x.Healthcheck } return nil } -func (m *EntrypointOptions) GetHook() *HookOptions { - if m != nil { - return m.Hook +func (x *EntrypointOptions) GetHook() *HookOptions { + if x != nil { + return x.Hook } return nil } -func (m *EntrypointOptions) GetRestartPolicy() string { - if m != nil { - return m.RestartPolicy +func (x *EntrypointOptions) GetRestartPolicy() string { + if x != nil { + return x.RestartPolicy } return "" } -func (m *EntrypointOptions) GetSysctls() map[string]string { - if m != nil { - return m.Sysctls +func (x *EntrypointOptions) GetSysctls() map[string]string { + if x != nil { + return x.Sysctls } return nil } -type DeployOptions struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Entrypoint *EntrypointOptions `protobuf:"bytes,2,opt,name=entrypoint,proto3" json:"entrypoint,omitempty"` - Podname string `protobuf:"bytes,3,opt,name=podname,proto3" json:"podname,omitempty"` - Nodenames []string `protobuf:"bytes,4,rep,name=nodenames,proto3" json:"nodenames,omitempty"` - Image string `protobuf:"bytes,5,opt,name=image,proto3" json:"image,omitempty"` - ExtraArgs string `protobuf:"bytes,6,opt,name=extra_args,json=extraArgs,proto3" json:"extra_args,omitempty"` - CpuQuota float64 `protobuf:"fixed64,7,opt,name=cpu_quota,json=cpuQuota,proto3" json:"cpu_quota,omitempty"` - Memory int64 `protobuf:"varint,8,opt,name=memory,proto3" json:"memory,omitempty"` - Count int32 `protobuf:"varint,9,opt,name=count,proto3" json:"count,omitempty"` - Env []string `protobuf:"bytes,10,rep,name=env,proto3" json:"env,omitempty"` - Dns []string `protobuf:"bytes,11,rep,name=dns,proto3" json:"dns,omitempty"` - ExtraHosts []string `protobuf:"bytes,12,rep,name=extra_hosts,json=extraHosts,proto3" json:"extra_hosts,omitempty"` - Volumes []string `protobuf:"bytes,13,rep,name=volumes,proto3" json:"volumes,omitempty"` - Networks map[string]string `protobuf:"bytes,14,rep,name=networks,proto3" json:"networks,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Networkmode string `protobuf:"bytes,15,opt,name=networkmode,proto3" json:"networkmode,omitempty"` - User string `protobuf:"bytes,16,opt,name=user,proto3" json:"user,omitempty"` - Debug bool `protobuf:"varint,17,opt,name=debug,proto3" json:"debug,omitempty"` - OpenStdin bool `protobuf:"varint,18,opt,name=openStdin,proto3" json:"openStdin,omitempty"` - Labels map[string]string `protobuf:"bytes,19,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Nodelabels map[string]string `protobuf:"bytes,20,rep,name=nodelabels,proto3" json:"nodelabels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - DeployStrategy DeployOptions_Strategy `protobuf:"varint,21,opt,name=deploy_strategy,json=deployStrategy,proto3,enum=pb.DeployOptions_Strategy" json:"deploy_strategy,omitempty"` - Data map[string][]byte `protobuf:"bytes,22,rep,name=data,proto3" json:"data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - SoftLimit bool `protobuf:"varint,23,opt,name=soft_limit,json=softLimit,proto3" json:"soft_limit,omitempty"` - NodesLimit int32 `protobuf:"varint,24,opt,name=nodes_limit,json=nodesLimit,proto3" json:"nodes_limit,omitempty"` - CpuBind bool `protobuf:"varint,25,opt,name=cpu_bind,json=cpuBind,proto3" json:"cpu_bind,omitempty"` - IgnoreHook bool `protobuf:"varint,26,opt,name=ignore_hook,json=ignoreHook,proto3" json:"ignore_hook,omitempty"` - AfterCreate []string `protobuf:"bytes,27,rep,name=after_create,json=afterCreate,proto3" json:"after_create,omitempty"` - RawArgs []byte `protobuf:"bytes,28,opt,name=raw_args,json=rawArgs,proto3" json:"raw_args,omitempty"` - Storage int64 `protobuf:"varint,29,opt,name=storage,proto3" json:"storage,omitempty"` - CpuQuotaRequest float64 `protobuf:"fixed64,30,opt,name=cpu_quota_request,json=cpuQuotaRequest,proto3" json:"cpu_quota_request,omitempty"` - MemoryRequest int64 `protobuf:"varint,31,opt,name=memory_request,json=memoryRequest,proto3" json:"memory_request,omitempty"` - VolumesRequest []string `protobuf:"bytes,32,rep,name=volumes_request,json=volumesRequest,proto3" json:"volumes_request,omitempty"` - StorageRequest int64 `protobuf:"varint,33,opt,name=storage_request,json=storageRequest,proto3" json:"storage_request,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *DeployOptions) Reset() { *m = DeployOptions{} } -func (m *DeployOptions) String() string { return proto.CompactTextString(m) } -func (*DeployOptions) ProtoMessage() {} -func (*DeployOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{44} +type ResourceOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CpuQuotaLimit float64 `protobuf:"fixed64,1,opt,name=cpu_quota_limit,json=cpuQuotaLimit,proto3" json:"cpu_quota_limit,omitempty"` + CpuQuotaRequest float64 `protobuf:"fixed64,2,opt,name=cpu_quota_request,json=cpuQuotaRequest,proto3" json:"cpu_quota_request,omitempty"` + CpuBind bool `protobuf:"varint,3,opt,name=cpu_bind,json=cpuBind,proto3" json:"cpu_bind,omitempty"` + MemoryLimit int64 `protobuf:"varint,4,opt,name=memory_limit,json=memoryLimit,proto3" json:"memory_limit,omitempty"` + MemoryRequest int64 `protobuf:"varint,5,opt,name=memory_request,json=memoryRequest,proto3" json:"memory_request,omitempty"` + StorageLimit int64 `protobuf:"varint,6,opt,name=storage_limit,json=storageLimit,proto3" json:"storage_limit,omitempty"` + StorageRequest int64 `protobuf:"varint,7,opt,name=storage_request,json=storageRequest,proto3" json:"storage_request,omitempty"` + VolumesLimit []string `protobuf:"bytes,8,rep,name=volumes_limit,json=volumesLimit,proto3" json:"volumes_limit,omitempty"` + VolumesRequest []string `protobuf:"bytes,9,rep,name=volumes_request,json=volumesRequest,proto3" json:"volumes_request,omitempty"` +} + +func (x *ResourceOptions) Reset() { + *x = ResourceOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResourceOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *DeployOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_DeployOptions.Unmarshal(m, b) +func (*ResourceOptions) ProtoMessage() {} + +func (x *ResourceOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[44] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -func (m *DeployOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_DeployOptions.Marshal(b, m, deterministic) + +// Deprecated: Use ResourceOptions.ProtoReflect.Descriptor instead. +func (*ResourceOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{44} } -func (m *DeployOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_DeployOptions.Merge(m, src) + +func (x *ResourceOptions) GetCpuQuotaLimit() float64 { + if x != nil { + return x.CpuQuotaLimit + } + return 0 } -func (m *DeployOptions) XXX_Size() int { - return xxx_messageInfo_DeployOptions.Size(m) + +func (x *ResourceOptions) GetCpuQuotaRequest() float64 { + if x != nil { + return x.CpuQuotaRequest + } + return 0 } -func (m *DeployOptions) XXX_DiscardUnknown() { - xxx_messageInfo_DeployOptions.DiscardUnknown(m) + +func (x *ResourceOptions) GetCpuBind() bool { + if x != nil { + return x.CpuBind + } + return false } -var xxx_messageInfo_DeployOptions proto.InternalMessageInfo +func (x *ResourceOptions) GetMemoryLimit() int64 { + if x != nil { + return x.MemoryLimit + } + return 0 +} -func (m *DeployOptions) GetName() string { - if m != nil { - return m.Name +func (x *ResourceOptions) GetMemoryRequest() int64 { + if x != nil { + return x.MemoryRequest } - return "" + return 0 } -func (m *DeployOptions) GetEntrypoint() *EntrypointOptions { - if m != nil { - return m.Entrypoint +func (x *ResourceOptions) GetStorageLimit() int64 { + if x != nil { + return x.StorageLimit } - return nil + return 0 } -func (m *DeployOptions) GetPodname() string { - if m != nil { - return m.Podname +func (x *ResourceOptions) GetStorageRequest() int64 { + if x != nil { + return x.StorageRequest } - return "" + return 0 } -func (m *DeployOptions) GetNodenames() []string { - if m != nil { - return m.Nodenames +func (x *ResourceOptions) GetVolumesLimit() []string { + if x != nil { + return x.VolumesLimit } return nil } -func (m *DeployOptions) GetImage() string { - if m != nil { - return m.Image +func (x *ResourceOptions) GetVolumesRequest() []string { + if x != nil { + return x.VolumesRequest } - return "" + return nil +} + +type Resource struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CpuQuotaLimit float64 `protobuf:"fixed64,1,opt,name=cpu_quota_limit,json=cpuQuotaLimit,proto3" json:"cpu_quota_limit,omitempty"` + CpuQuotaRequest float64 `protobuf:"fixed64,2,opt,name=cpu_quota_request,json=cpuQuotaRequest,proto3" json:"cpu_quota_request,omitempty"` + Cpu map[string]int32 `protobuf:"bytes,3,rep,name=cpu,proto3" json:"cpu,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + MemoryLimit int64 `protobuf:"varint,4,opt,name=memory_limit,json=memoryLimit,proto3" json:"memory_limit,omitempty"` + MemoryRequest int64 `protobuf:"varint,5,opt,name=memory_request,json=memoryRequest,proto3" json:"memory_request,omitempty"` + StorageLimit int64 `protobuf:"varint,6,opt,name=storage_limit,json=storageLimit,proto3" json:"storage_limit,omitempty"` + StorageRequest int64 `protobuf:"varint,7,opt,name=storage_request,json=storageRequest,proto3" json:"storage_request,omitempty"` + VolumesLimit []string `protobuf:"bytes,8,rep,name=volumes_limit,json=volumesLimit,proto3" json:"volumes_limit,omitempty"` + VolumesRequest []string `protobuf:"bytes,9,rep,name=volumes_request,json=volumesRequest,proto3" json:"volumes_request,omitempty"` + VolumePlanLimit map[string]*Volume `protobuf:"bytes,10,rep,name=volume_plan_limit,json=volumePlanLimit,proto3" json:"volume_plan_limit,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + VolumePlanRequest map[string]*Volume `protobuf:"bytes,11,rep,name=volume_plan_request,json=volumePlanRequest,proto3" json:"volume_plan_request,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (m *DeployOptions) GetExtraArgs() string { - if m != nil { - return m.ExtraArgs +func (x *Resource) Reset() { + *x = Resource{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } - return "" } -func (m *DeployOptions) GetCpuQuota() float64 { - if m != nil { - return m.CpuQuota +func (x *Resource) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Resource) ProtoMessage() {} + +func (x *Resource) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[45] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Resource.ProtoReflect.Descriptor instead. +func (*Resource) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{45} +} + +func (x *Resource) GetCpuQuotaLimit() float64 { + if x != nil { + return x.CpuQuotaLimit + } + return 0 +} + +func (x *Resource) GetCpuQuotaRequest() float64 { + if x != nil { + return x.CpuQuotaRequest + } + return 0 +} + +func (x *Resource) GetCpu() map[string]int32 { + if x != nil { + return x.Cpu + } + return nil +} + +func (x *Resource) GetMemoryLimit() int64 { + if x != nil { + return x.MemoryLimit + } + return 0 +} + +func (x *Resource) GetMemoryRequest() int64 { + if x != nil { + return x.MemoryRequest } return 0 } -func (m *DeployOptions) GetMemory() int64 { - if m != nil { - return m.Memory +func (x *Resource) GetStorageLimit() int64 { + if x != nil { + return x.StorageLimit } return 0 } -func (m *DeployOptions) GetCount() int32 { - if m != nil { - return m.Count +func (x *Resource) GetStorageRequest() int64 { + if x != nil { + return x.StorageRequest } return 0 } -func (m *DeployOptions) GetEnv() []string { - if m != nil { - return m.Env +func (x *Resource) GetVolumesLimit() []string { + if x != nil { + return x.VolumesLimit } return nil } -func (m *DeployOptions) GetDns() []string { - if m != nil { - return m.Dns +func (x *Resource) GetVolumesRequest() []string { + if x != nil { + return x.VolumesRequest } return nil } -func (m *DeployOptions) GetExtraHosts() []string { - if m != nil { - return m.ExtraHosts +func (x *Resource) GetVolumePlanLimit() map[string]*Volume { + if x != nil { + return x.VolumePlanLimit } return nil } -func (m *DeployOptions) GetVolumes() []string { - if m != nil { - return m.Volumes +func (x *Resource) GetVolumePlanRequest() map[string]*Volume { + if x != nil { + return x.VolumePlanRequest } return nil } -func (m *DeployOptions) GetNetworks() map[string]string { - if m != nil { - return m.Networks +type Volume struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Volume map[string]int64 `protobuf:"bytes,1,rep,name=volume,proto3" json:"volume,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` +} + +func (x *Volume) Reset() { + *x = Volume{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Volume) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Volume) ProtoMessage() {} + +func (x *Volume) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[46] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Volume.ProtoReflect.Descriptor instead. +func (*Volume) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{46} +} + +func (x *Volume) GetVolume() map[string]int64 { + if x != nil { + return x.Volume } return nil } -func (m *DeployOptions) GetNetworkmode() string { - if m != nil { - return m.Networkmode +type DeployOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Entrypoint *EntrypointOptions `protobuf:"bytes,2,opt,name=entrypoint,proto3" json:"entrypoint,omitempty"` + Podname string `protobuf:"bytes,3,opt,name=podname,proto3" json:"podname,omitempty"` + Nodenames []string `protobuf:"bytes,4,rep,name=nodenames,proto3" json:"nodenames,omitempty"` + Image string `protobuf:"bytes,5,opt,name=image,proto3" json:"image,omitempty"` + ExtraArgs string `protobuf:"bytes,6,opt,name=extra_args,json=extraArgs,proto3" json:"extra_args,omitempty"` + Count int32 `protobuf:"varint,7,opt,name=count,proto3" json:"count,omitempty"` + Env []string `protobuf:"bytes,8,rep,name=env,proto3" json:"env,omitempty"` + Dns []string `protobuf:"bytes,9,rep,name=dns,proto3" json:"dns,omitempty"` + ExtraHosts []string `protobuf:"bytes,10,rep,name=extra_hosts,json=extraHosts,proto3" json:"extra_hosts,omitempty"` + Networks map[string]string `protobuf:"bytes,11,rep,name=networks,proto3" json:"networks,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Networkmode string `protobuf:"bytes,12,opt,name=networkmode,proto3" json:"networkmode,omitempty"` + User string `protobuf:"bytes,13,opt,name=user,proto3" json:"user,omitempty"` + Debug bool `protobuf:"varint,14,opt,name=debug,proto3" json:"debug,omitempty"` + OpenStdin bool `protobuf:"varint,15,opt,name=openStdin,proto3" json:"openStdin,omitempty"` + Labels map[string]string `protobuf:"bytes,16,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Nodelabels map[string]string `protobuf:"bytes,17,rep,name=nodelabels,proto3" json:"nodelabels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + DeployStrategy DeployOptions_Strategy `protobuf:"varint,18,opt,name=deploy_strategy,json=deployStrategy,proto3,enum=pb.DeployOptions_Strategy" json:"deploy_strategy,omitempty"` + Data map[string][]byte `protobuf:"bytes,19,rep,name=data,proto3" json:"data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + NodesLimit int32 `protobuf:"varint,20,opt,name=nodes_limit,json=nodesLimit,proto3" json:"nodes_limit,omitempty"` + IgnoreHook bool `protobuf:"varint,21,opt,name=ignore_hook,json=ignoreHook,proto3" json:"ignore_hook,omitempty"` + AfterCreate []string `protobuf:"bytes,22,rep,name=after_create,json=afterCreate,proto3" json:"after_create,omitempty"` + RawArgs []byte `protobuf:"bytes,23,opt,name=raw_args,json=rawArgs,proto3" json:"raw_args,omitempty"` + ResourceOpts *ResourceOptions `protobuf:"bytes,24,opt,name=resource_opts,json=resourceOpts,proto3" json:"resource_opts,omitempty"` +} + +func (x *DeployOptions) Reset() { + *x = DeployOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeployOptions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeployOptions) ProtoMessage() {} + +func (x *DeployOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[47] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeployOptions.ProtoReflect.Descriptor instead. +func (*DeployOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{47} +} + +func (x *DeployOptions) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *DeployOptions) GetUser() string { - if m != nil { - return m.User +func (x *DeployOptions) GetEntrypoint() *EntrypointOptions { + if x != nil { + return x.Entrypoint + } + return nil +} + +func (x *DeployOptions) GetPodname() string { + if x != nil { + return x.Podname } return "" } -func (m *DeployOptions) GetDebug() bool { - if m != nil { - return m.Debug +func (x *DeployOptions) GetNodenames() []string { + if x != nil { + return x.Nodenames } - return false + return nil } -func (m *DeployOptions) GetOpenStdin() bool { - if m != nil { - return m.OpenStdin +func (x *DeployOptions) GetImage() string { + if x != nil { + return x.Image } - return false + return "" +} + +func (x *DeployOptions) GetExtraArgs() string { + if x != nil { + return x.ExtraArgs + } + return "" +} + +func (x *DeployOptions) GetCount() int32 { + if x != nil { + return x.Count + } + return 0 } -func (m *DeployOptions) GetLabels() map[string]string { - if m != nil { - return m.Labels +func (x *DeployOptions) GetEnv() []string { + if x != nil { + return x.Env } return nil } -func (m *DeployOptions) GetNodelabels() map[string]string { - if m != nil { - return m.Nodelabels +func (x *DeployOptions) GetDns() []string { + if x != nil { + return x.Dns } return nil } -func (m *DeployOptions) GetDeployStrategy() DeployOptions_Strategy { - if m != nil { - return m.DeployStrategy +func (x *DeployOptions) GetExtraHosts() []string { + if x != nil { + return x.ExtraHosts } - return DeployOptions_AUTO + return nil } -func (m *DeployOptions) GetData() map[string][]byte { - if m != nil { - return m.Data +func (x *DeployOptions) GetNetworks() map[string]string { + if x != nil { + return x.Networks } return nil } -func (m *DeployOptions) GetSoftLimit() bool { - if m != nil { - return m.SoftLimit +func (x *DeployOptions) GetNetworkmode() string { + if x != nil { + return x.Networkmode } - return false + return "" } -func (m *DeployOptions) GetNodesLimit() int32 { - if m != nil { - return m.NodesLimit +func (x *DeployOptions) GetUser() string { + if x != nil { + return x.User } - return 0 + return "" } -func (m *DeployOptions) GetCpuBind() bool { - if m != nil { - return m.CpuBind +func (x *DeployOptions) GetDebug() bool { + if x != nil { + return x.Debug } return false } -func (m *DeployOptions) GetIgnoreHook() bool { - if m != nil { - return m.IgnoreHook +func (x *DeployOptions) GetOpenStdin() bool { + if x != nil { + return x.OpenStdin } return false } -func (m *DeployOptions) GetAfterCreate() []string { - if m != nil { - return m.AfterCreate +func (x *DeployOptions) GetLabels() map[string]string { + if x != nil { + return x.Labels } return nil } -func (m *DeployOptions) GetRawArgs() []byte { - if m != nil { - return m.RawArgs +func (x *DeployOptions) GetNodelabels() map[string]string { + if x != nil { + return x.Nodelabels } return nil } -func (m *DeployOptions) GetStorage() int64 { - if m != nil { - return m.Storage +func (x *DeployOptions) GetDeployStrategy() DeployOptions_Strategy { + if x != nil { + return x.DeployStrategy } - return 0 + return DeployOptions_AUTO } -func (m *DeployOptions) GetCpuQuotaRequest() float64 { - if m != nil { - return m.CpuQuotaRequest +func (x *DeployOptions) GetData() map[string][]byte { + if x != nil { + return x.Data } - return 0 + return nil } -func (m *DeployOptions) GetMemoryRequest() int64 { - if m != nil { - return m.MemoryRequest +func (x *DeployOptions) GetNodesLimit() int32 { + if x != nil { + return x.NodesLimit } return 0 } -func (m *DeployOptions) GetVolumesRequest() []string { - if m != nil { - return m.VolumesRequest +func (x *DeployOptions) GetIgnoreHook() bool { + if x != nil { + return x.IgnoreHook } - return nil + return false } -func (m *DeployOptions) GetStorageRequest() int64 { - if m != nil { - return m.StorageRequest +func (x *DeployOptions) GetAfterCreate() []string { + if x != nil { + return x.AfterCreate } - return 0 + return nil } -type ReplaceOptions struct { - DeployOpt *DeployOptions `protobuf:"bytes,1,opt,name=deployOpt,proto3" json:"deployOpt,omitempty"` - Networkinherit bool `protobuf:"varint,2,opt,name=networkinherit,proto3" json:"networkinherit,omitempty"` - FilterLabels map[string]string `protobuf:"bytes,3,rep,name=filter_labels,json=filterLabels,proto3" json:"filter_labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Copy map[string]string `protobuf:"bytes,4,rep,name=copy,proto3" json:"copy,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Ids []string `protobuf:"bytes,5,rep,name=ids,proto3" json:"ids,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ReplaceOptions) Reset() { *m = ReplaceOptions{} } -func (m *ReplaceOptions) String() string { return proto.CompactTextString(m) } -func (*ReplaceOptions) ProtoMessage() {} -func (*ReplaceOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{45} +func (x *DeployOptions) GetRawArgs() []byte { + if x != nil { + return x.RawArgs + } + return nil } -func (m *ReplaceOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ReplaceOptions.Unmarshal(m, b) +func (x *DeployOptions) GetResourceOpts() *ResourceOptions { + if x != nil { + return x.ResourceOpts + } + return nil } -func (m *ReplaceOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ReplaceOptions.Marshal(b, m, deterministic) + +type ReplaceOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + DeployOpt *DeployOptions `protobuf:"bytes,1,opt,name=deployOpt,proto3" json:"deployOpt,omitempty"` + Networkinherit bool `protobuf:"varint,2,opt,name=networkinherit,proto3" json:"networkinherit,omitempty"` + FilterLabels map[string]string `protobuf:"bytes,3,rep,name=filter_labels,json=filterLabels,proto3" json:"filter_labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Copy map[string]string `protobuf:"bytes,4,rep,name=copy,proto3" json:"copy,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Ids []string `protobuf:"bytes,5,rep,name=ids,proto3" json:"ids,omitempty"` } -func (m *ReplaceOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_ReplaceOptions.Merge(m, src) + +func (x *ReplaceOptions) Reset() { + *x = ReplaceOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ReplaceOptions) XXX_Size() int { - return xxx_messageInfo_ReplaceOptions.Size(m) + +func (x *ReplaceOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ReplaceOptions) XXX_DiscardUnknown() { - xxx_messageInfo_ReplaceOptions.DiscardUnknown(m) + +func (*ReplaceOptions) ProtoMessage() {} + +func (x *ReplaceOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[48] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ReplaceOptions proto.InternalMessageInfo +// Deprecated: Use ReplaceOptions.ProtoReflect.Descriptor instead. +func (*ReplaceOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{48} +} -func (m *ReplaceOptions) GetDeployOpt() *DeployOptions { - if m != nil { - return m.DeployOpt +func (x *ReplaceOptions) GetDeployOpt() *DeployOptions { + if x != nil { + return x.DeployOpt } return nil } -func (m *ReplaceOptions) GetNetworkinherit() bool { - if m != nil { - return m.Networkinherit +func (x *ReplaceOptions) GetNetworkinherit() bool { + if x != nil { + return x.Networkinherit } return false } -func (m *ReplaceOptions) GetFilterLabels() map[string]string { - if m != nil { - return m.FilterLabels +func (x *ReplaceOptions) GetFilterLabels() map[string]string { + if x != nil { + return x.FilterLabels } return nil } -func (m *ReplaceOptions) GetCopy() map[string]string { - if m != nil { - return m.Copy +func (x *ReplaceOptions) GetCopy() map[string]string { + if x != nil { + return x.Copy } return nil } -func (m *ReplaceOptions) GetIds() []string { - if m != nil { - return m.Ids +func (x *ReplaceOptions) GetIds() []string { + if x != nil { + return x.Ids } return nil } type CacheImageOptions struct { - Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` - Nodenames []string `protobuf:"bytes,2,rep,name=nodenames,proto3" json:"nodenames,omitempty"` - Images []string `protobuf:"bytes,3,rep,name=images,proto3" json:"images,omitempty"` - Step int32 `protobuf:"varint,4,opt,name=step,proto3" json:"step,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CacheImageOptions) Reset() { *m = CacheImageOptions{} } -func (m *CacheImageOptions) String() string { return proto.CompactTextString(m) } -func (*CacheImageOptions) ProtoMessage() {} -func (*CacheImageOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{46} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *CacheImageOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CacheImageOptions.Unmarshal(m, b) + Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` + Nodenames []string `protobuf:"bytes,2,rep,name=nodenames,proto3" json:"nodenames,omitempty"` + Images []string `protobuf:"bytes,3,rep,name=images,proto3" json:"images,omitempty"` + Step int32 `protobuf:"varint,4,opt,name=step,proto3" json:"step,omitempty"` } -func (m *CacheImageOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CacheImageOptions.Marshal(b, m, deterministic) -} -func (m *CacheImageOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_CacheImageOptions.Merge(m, src) + +func (x *CacheImageOptions) Reset() { + *x = CacheImageOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *CacheImageOptions) XXX_Size() int { - return xxx_messageInfo_CacheImageOptions.Size(m) + +func (x *CacheImageOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *CacheImageOptions) XXX_DiscardUnknown() { - xxx_messageInfo_CacheImageOptions.DiscardUnknown(m) + +func (*CacheImageOptions) ProtoMessage() {} + +func (x *CacheImageOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[49] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_CacheImageOptions proto.InternalMessageInfo +// Deprecated: Use CacheImageOptions.ProtoReflect.Descriptor instead. +func (*CacheImageOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{49} +} -func (m *CacheImageOptions) GetPodname() string { - if m != nil { - return m.Podname +func (x *CacheImageOptions) GetPodname() string { + if x != nil { + return x.Podname } return "" } -func (m *CacheImageOptions) GetNodenames() []string { - if m != nil { - return m.Nodenames +func (x *CacheImageOptions) GetNodenames() []string { + if x != nil { + return x.Nodenames } return nil } -func (m *CacheImageOptions) GetImages() []string { - if m != nil { - return m.Images +func (x *CacheImageOptions) GetImages() []string { + if x != nil { + return x.Images } return nil } -func (m *CacheImageOptions) GetStep() int32 { - if m != nil { - return m.Step +func (x *CacheImageOptions) GetStep() int32 { + if x != nil { + return x.Step } return 0 } type RemoveImageOptions struct { - Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` - Nodenames []string `protobuf:"bytes,2,rep,name=nodenames,proto3" json:"nodenames,omitempty"` - Images []string `protobuf:"bytes,3,rep,name=images,proto3" json:"images,omitempty"` - Step int32 `protobuf:"varint,4,opt,name=step,proto3" json:"step,omitempty"` - Prune bool `protobuf:"varint,5,opt,name=prune,proto3" json:"prune,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *RemoveImageOptions) Reset() { *m = RemoveImageOptions{} } -func (m *RemoveImageOptions) String() string { return proto.CompactTextString(m) } -func (*RemoveImageOptions) ProtoMessage() {} -func (*RemoveImageOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{47} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *RemoveImageOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RemoveImageOptions.Unmarshal(m, b) -} -func (m *RemoveImageOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RemoveImageOptions.Marshal(b, m, deterministic) + Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` + Nodenames []string `protobuf:"bytes,2,rep,name=nodenames,proto3" json:"nodenames,omitempty"` + Images []string `protobuf:"bytes,3,rep,name=images,proto3" json:"images,omitempty"` + Step int32 `protobuf:"varint,4,opt,name=step,proto3" json:"step,omitempty"` + Prune bool `protobuf:"varint,5,opt,name=prune,proto3" json:"prune,omitempty"` } -func (m *RemoveImageOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_RemoveImageOptions.Merge(m, src) + +func (x *RemoveImageOptions) Reset() { + *x = RemoveImageOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *RemoveImageOptions) XXX_Size() int { - return xxx_messageInfo_RemoveImageOptions.Size(m) + +func (x *RemoveImageOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *RemoveImageOptions) XXX_DiscardUnknown() { - xxx_messageInfo_RemoveImageOptions.DiscardUnknown(m) + +func (*RemoveImageOptions) ProtoMessage() {} + +func (x *RemoveImageOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[50] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_RemoveImageOptions proto.InternalMessageInfo +// Deprecated: Use RemoveImageOptions.ProtoReflect.Descriptor instead. +func (*RemoveImageOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{50} +} -func (m *RemoveImageOptions) GetPodname() string { - if m != nil { - return m.Podname +func (x *RemoveImageOptions) GetPodname() string { + if x != nil { + return x.Podname } return "" } -func (m *RemoveImageOptions) GetNodenames() []string { - if m != nil { - return m.Nodenames +func (x *RemoveImageOptions) GetNodenames() []string { + if x != nil { + return x.Nodenames } return nil } -func (m *RemoveImageOptions) GetImages() []string { - if m != nil { - return m.Images +func (x *RemoveImageOptions) GetImages() []string { + if x != nil { + return x.Images } return nil } -func (m *RemoveImageOptions) GetStep() int32 { - if m != nil { - return m.Step +func (x *RemoveImageOptions) GetStep() int32 { + if x != nil { + return x.Step } return 0 } -func (m *RemoveImageOptions) GetPrune() bool { - if m != nil { - return m.Prune +func (x *RemoveImageOptions) GetPrune() bool { + if x != nil { + return x.Prune } return false } type CopyPaths struct { - Paths []string `protobuf:"bytes,1,rep,name=paths,proto3" json:"paths,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *CopyPaths) Reset() { *m = CopyPaths{} } -func (m *CopyPaths) String() string { return proto.CompactTextString(m) } -func (*CopyPaths) ProtoMessage() {} -func (*CopyPaths) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{48} + Paths []string `protobuf:"bytes,1,rep,name=paths,proto3" json:"paths,omitempty"` } -func (m *CopyPaths) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CopyPaths.Unmarshal(m, b) -} -func (m *CopyPaths) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CopyPaths.Marshal(b, m, deterministic) -} -func (m *CopyPaths) XXX_Merge(src proto.Message) { - xxx_messageInfo_CopyPaths.Merge(m, src) +func (x *CopyPaths) Reset() { + *x = CopyPaths{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *CopyPaths) XXX_Size() int { - return xxx_messageInfo_CopyPaths.Size(m) + +func (x *CopyPaths) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *CopyPaths) XXX_DiscardUnknown() { - xxx_messageInfo_CopyPaths.DiscardUnknown(m) + +func (*CopyPaths) ProtoMessage() {} + +func (x *CopyPaths) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[51] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_CopyPaths proto.InternalMessageInfo +// Deprecated: Use CopyPaths.ProtoReflect.Descriptor instead. +func (*CopyPaths) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{51} +} -func (m *CopyPaths) GetPaths() []string { - if m != nil { - return m.Paths +func (x *CopyPaths) GetPaths() []string { + if x != nil { + return x.Paths } return nil } type CopyOptions struct { - Targets map[string]*CopyPaths `protobuf:"bytes,1,rep,name=targets,proto3" json:"targets,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *CopyOptions) Reset() { *m = CopyOptions{} } -func (m *CopyOptions) String() string { return proto.CompactTextString(m) } -func (*CopyOptions) ProtoMessage() {} -func (*CopyOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{49} + Targets map[string]*CopyPaths `protobuf:"bytes,1,rep,name=targets,proto3" json:"targets,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (m *CopyOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CopyOptions.Unmarshal(m, b) -} -func (m *CopyOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CopyOptions.Marshal(b, m, deterministic) -} -func (m *CopyOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_CopyOptions.Merge(m, src) +func (x *CopyOptions) Reset() { + *x = CopyOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *CopyOptions) XXX_Size() int { - return xxx_messageInfo_CopyOptions.Size(m) + +func (x *CopyOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *CopyOptions) XXX_DiscardUnknown() { - xxx_messageInfo_CopyOptions.DiscardUnknown(m) + +func (*CopyOptions) ProtoMessage() {} + +func (x *CopyOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[52] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_CopyOptions proto.InternalMessageInfo +// Deprecated: Use CopyOptions.ProtoReflect.Descriptor instead. +func (*CopyOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{52} +} -func (m *CopyOptions) GetTargets() map[string]*CopyPaths { - if m != nil { - return m.Targets +func (x *CopyOptions) GetTargets() map[string]*CopyPaths { + if x != nil { + return x.Targets } return nil } type SendOptions struct { - Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` - Data map[string][]byte `protobuf:"bytes,2,rep,name=data,proto3" json:"data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *SendOptions) Reset() { *m = SendOptions{} } -func (m *SendOptions) String() string { return proto.CompactTextString(m) } -func (*SendOptions) ProtoMessage() {} -func (*SendOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{50} + Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` + Data map[string][]byte `protobuf:"bytes,2,rep,name=data,proto3" json:"data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } -func (m *SendOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SendOptions.Unmarshal(m, b) -} -func (m *SendOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SendOptions.Marshal(b, m, deterministic) -} -func (m *SendOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_SendOptions.Merge(m, src) +func (x *SendOptions) Reset() { + *x = SendOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *SendOptions) XXX_Size() int { - return xxx_messageInfo_SendOptions.Size(m) + +func (x *SendOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *SendOptions) XXX_DiscardUnknown() { - xxx_messageInfo_SendOptions.DiscardUnknown(m) + +func (*SendOptions) ProtoMessage() {} + +func (x *SendOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[53] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_SendOptions proto.InternalMessageInfo +// Deprecated: Use SendOptions.ProtoReflect.Descriptor instead. +func (*SendOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{53} +} -func (m *SendOptions) GetIds() []string { - if m != nil { - return m.Ids +func (x *SendOptions) GetIds() []string { + if x != nil { + return x.Ids } return nil } -func (m *SendOptions) GetData() map[string][]byte { - if m != nil { - return m.Data +func (x *SendOptions) GetData() map[string][]byte { + if x != nil { + return x.Data } return nil } type ErrorDetail struct { - Code int64 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` - Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ErrorDetail) Reset() { *m = ErrorDetail{} } -func (m *ErrorDetail) String() string { return proto.CompactTextString(m) } -func (*ErrorDetail) ProtoMessage() {} -func (*ErrorDetail) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{51} + Code int64 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` + Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` } -func (m *ErrorDetail) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ErrorDetail.Unmarshal(m, b) -} -func (m *ErrorDetail) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ErrorDetail.Marshal(b, m, deterministic) -} -func (m *ErrorDetail) XXX_Merge(src proto.Message) { - xxx_messageInfo_ErrorDetail.Merge(m, src) +func (x *ErrorDetail) Reset() { + *x = ErrorDetail{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ErrorDetail) XXX_Size() int { - return xxx_messageInfo_ErrorDetail.Size(m) + +func (x *ErrorDetail) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ErrorDetail) XXX_DiscardUnknown() { - xxx_messageInfo_ErrorDetail.DiscardUnknown(m) + +func (*ErrorDetail) ProtoMessage() {} + +func (x *ErrorDetail) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[54] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ErrorDetail proto.InternalMessageInfo +// Deprecated: Use ErrorDetail.ProtoReflect.Descriptor instead. +func (*ErrorDetail) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{54} +} -func (m *ErrorDetail) GetCode() int64 { - if m != nil { - return m.Code +func (x *ErrorDetail) GetCode() int64 { + if x != nil { + return x.Code } return 0 } -func (m *ErrorDetail) GetMessage() string { - if m != nil { - return m.Message +func (x *ErrorDetail) GetMessage() string { + if x != nil { + return x.Message } return "" } type BuildImageMessage struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Status string `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - Progress string `protobuf:"bytes,3,opt,name=progress,proto3" json:"progress,omitempty"` - Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"` - Stream string `protobuf:"bytes,5,opt,name=stream,proto3" json:"stream,omitempty"` - ErrorDetail *ErrorDetail `protobuf:"bytes,6,opt,name=error_detail,json=errorDetail,proto3" json:"error_detail,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *BuildImageMessage) Reset() { *m = BuildImageMessage{} } -func (m *BuildImageMessage) String() string { return proto.CompactTextString(m) } -func (*BuildImageMessage) ProtoMessage() {} -func (*BuildImageMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{52} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *BuildImageMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_BuildImageMessage.Unmarshal(m, b) + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Status string `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` + Progress string `protobuf:"bytes,3,opt,name=progress,proto3" json:"progress,omitempty"` + Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"` + Stream string `protobuf:"bytes,5,opt,name=stream,proto3" json:"stream,omitempty"` + ErrorDetail *ErrorDetail `protobuf:"bytes,6,opt,name=error_detail,json=errorDetail,proto3" json:"error_detail,omitempty"` } -func (m *BuildImageMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_BuildImageMessage.Marshal(b, m, deterministic) -} -func (m *BuildImageMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_BuildImageMessage.Merge(m, src) + +func (x *BuildImageMessage) Reset() { + *x = BuildImageMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *BuildImageMessage) XXX_Size() int { - return xxx_messageInfo_BuildImageMessage.Size(m) + +func (x *BuildImageMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *BuildImageMessage) XXX_DiscardUnknown() { - xxx_messageInfo_BuildImageMessage.DiscardUnknown(m) + +func (*BuildImageMessage) ProtoMessage() {} + +func (x *BuildImageMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[55] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_BuildImageMessage proto.InternalMessageInfo +// Deprecated: Use BuildImageMessage.ProtoReflect.Descriptor instead. +func (*BuildImageMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{55} +} -func (m *BuildImageMessage) GetId() string { - if m != nil { - return m.Id +func (x *BuildImageMessage) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *BuildImageMessage) GetStatus() string { - if m != nil { - return m.Status +func (x *BuildImageMessage) GetStatus() string { + if x != nil { + return x.Status } return "" } -func (m *BuildImageMessage) GetProgress() string { - if m != nil { - return m.Progress +func (x *BuildImageMessage) GetProgress() string { + if x != nil { + return x.Progress } return "" } -func (m *BuildImageMessage) GetError() string { - if m != nil { - return m.Error +func (x *BuildImageMessage) GetError() string { + if x != nil { + return x.Error } return "" } -func (m *BuildImageMessage) GetStream() string { - if m != nil { - return m.Stream +func (x *BuildImageMessage) GetStream() string { + if x != nil { + return x.Stream } return "" } -func (m *BuildImageMessage) GetErrorDetail() *ErrorDetail { - if m != nil { - return m.ErrorDetail +func (x *BuildImageMessage) GetErrorDetail() *ErrorDetail { + if x != nil { + return x.ErrorDetail } return nil } -type Volume struct { - Volume map[string]int64 `protobuf:"bytes,1,rep,name=volume,proto3" json:"volume,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} +type CreateContainerMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *Volume) Reset() { *m = Volume{} } -func (m *Volume) String() string { return proto.CompactTextString(m) } -func (*Volume) ProtoMessage() {} -func (*Volume) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{53} + Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` + Nodename string `protobuf:"bytes,2,opt,name=nodename,proto3" json:"nodename,omitempty"` + Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"` + Name string `protobuf:"bytes,4,opt,name=name,proto3" json:"name,omitempty"` + Error string `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"` + Success bool `protobuf:"varint,6,opt,name=success,proto3" json:"success,omitempty"` + Publish map[string]string `protobuf:"bytes,7,rep,name=publish,proto3" json:"publish,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Hook []byte `protobuf:"bytes,8,opt,name=hook,proto3" json:"hook,omitempty"` + Resource *Resource `protobuf:"bytes,9,opt,name=resource,proto3" json:"resource,omitempty"` } -func (m *Volume) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Volume.Unmarshal(m, b) -} -func (m *Volume) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Volume.Marshal(b, m, deterministic) -} -func (m *Volume) XXX_Merge(src proto.Message) { - xxx_messageInfo_Volume.Merge(m, src) -} -func (m *Volume) XXX_Size() int { - return xxx_messageInfo_Volume.Size(m) +func (x *CreateContainerMessage) Reset() { + *x = CreateContainerMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *Volume) XXX_DiscardUnknown() { - xxx_messageInfo_Volume.DiscardUnknown(m) + +func (x *CreateContainerMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -var xxx_messageInfo_Volume proto.InternalMessageInfo +func (*CreateContainerMessage) ProtoMessage() {} -func (m *Volume) GetVolume() map[string]int64 { - if m != nil { - return m.Volume +func (x *CreateContainerMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[56] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } - return nil + return mi.MessageOf(x) } -type CreateContainerMessage struct { - Podname string `protobuf:"bytes,1,opt,name=podname,proto3" json:"podname,omitempty"` - Nodename string `protobuf:"bytes,2,opt,name=nodename,proto3" json:"nodename,omitempty"` - Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"` - Name string `protobuf:"bytes,4,opt,name=name,proto3" json:"name,omitempty"` - Error string `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"` - Success bool `protobuf:"varint,6,opt,name=success,proto3" json:"success,omitempty"` - Cpu map[string]int32 `protobuf:"bytes,7,rep,name=cpu,proto3" json:"cpu,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Quota float64 `protobuf:"fixed64,8,opt,name=quota,proto3" json:"quota,omitempty"` - Memory int64 `protobuf:"varint,9,opt,name=memory,proto3" json:"memory,omitempty"` - Publish map[string]string `protobuf:"bytes,10,rep,name=publish,proto3" json:"publish,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Hook []byte `protobuf:"bytes,11,opt,name=hook,proto3" json:"hook,omitempty"` - Storage int64 `protobuf:"varint,12,opt,name=storage,proto3" json:"storage,omitempty"` - VolumePlan map[string]*Volume `protobuf:"bytes,13,rep,name=volume_plan,json=volumePlan,proto3" json:"volume_plan,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - QuotaRequest float64 `protobuf:"fixed64,14,opt,name=quota_request,json=quotaRequest,proto3" json:"quota_request,omitempty"` - MemoryRequest int64 `protobuf:"varint,15,opt,name=memory_request,json=memoryRequest,proto3" json:"memory_request,omitempty"` - StorageRequest int64 `protobuf:"varint,16,opt,name=storage_request,json=storageRequest,proto3" json:"storage_request,omitempty"` - VolumePlanRequest map[string]*Volume `protobuf:"bytes,17,rep,name=volume_plan_request,json=volumePlanRequest,proto3" json:"volume_plan_request,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CreateContainerMessage) Reset() { *m = CreateContainerMessage{} } -func (m *CreateContainerMessage) String() string { return proto.CompactTextString(m) } -func (*CreateContainerMessage) ProtoMessage() {} +// Deprecated: Use CreateContainerMessage.ProtoReflect.Descriptor instead. func (*CreateContainerMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{54} -} - -func (m *CreateContainerMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CreateContainerMessage.Unmarshal(m, b) -} -func (m *CreateContainerMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CreateContainerMessage.Marshal(b, m, deterministic) -} -func (m *CreateContainerMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_CreateContainerMessage.Merge(m, src) -} -func (m *CreateContainerMessage) XXX_Size() int { - return xxx_messageInfo_CreateContainerMessage.Size(m) + return file_core_proto_rawDescGZIP(), []int{56} } -func (m *CreateContainerMessage) XXX_DiscardUnknown() { - xxx_messageInfo_CreateContainerMessage.DiscardUnknown(m) -} - -var xxx_messageInfo_CreateContainerMessage proto.InternalMessageInfo -func (m *CreateContainerMessage) GetPodname() string { - if m != nil { - return m.Podname +func (x *CreateContainerMessage) GetPodname() string { + if x != nil { + return x.Podname } return "" } -func (m *CreateContainerMessage) GetNodename() string { - if m != nil { - return m.Nodename +func (x *CreateContainerMessage) GetNodename() string { + if x != nil { + return x.Nodename } return "" } -func (m *CreateContainerMessage) GetId() string { - if m != nil { - return m.Id +func (x *CreateContainerMessage) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *CreateContainerMessage) GetName() string { - if m != nil { - return m.Name +func (x *CreateContainerMessage) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *CreateContainerMessage) GetError() string { - if m != nil { - return m.Error +func (x *CreateContainerMessage) GetError() string { + if x != nil { + return x.Error } return "" } -func (m *CreateContainerMessage) GetSuccess() bool { - if m != nil { - return m.Success +func (x *CreateContainerMessage) GetSuccess() bool { + if x != nil { + return x.Success } return false } -func (m *CreateContainerMessage) GetCpu() map[string]int32 { - if m != nil { - return m.Cpu +func (x *CreateContainerMessage) GetPublish() map[string]string { + if x != nil { + return x.Publish } return nil } -func (m *CreateContainerMessage) GetQuota() float64 { - if m != nil { - return m.Quota - } - return 0 -} - -func (m *CreateContainerMessage) GetMemory() int64 { - if m != nil { - return m.Memory - } - return 0 -} - -func (m *CreateContainerMessage) GetPublish() map[string]string { - if m != nil { - return m.Publish +func (x *CreateContainerMessage) GetHook() []byte { + if x != nil { + return x.Hook } return nil } -func (m *CreateContainerMessage) GetHook() []byte { - if m != nil { - return m.Hook +func (x *CreateContainerMessage) GetResource() *Resource { + if x != nil { + return x.Resource } return nil } -func (m *CreateContainerMessage) GetStorage() int64 { - if m != nil { - return m.Storage - } - return 0 -} +type ReplaceContainerMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *CreateContainerMessage) GetVolumePlan() map[string]*Volume { - if m != nil { - return m.VolumePlan - } - return nil + Create *CreateContainerMessage `protobuf:"bytes,1,opt,name=create,proto3" json:"create,omitempty"` + Remove *RemoveContainerMessage `protobuf:"bytes,2,opt,name=remove,proto3" json:"remove,omitempty"` + Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` } -func (m *CreateContainerMessage) GetQuotaRequest() float64 { - if m != nil { - return m.QuotaRequest +func (x *ReplaceContainerMessage) Reset() { + *x = ReplaceContainerMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[57] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } - return 0 } -func (m *CreateContainerMessage) GetMemoryRequest() int64 { - if m != nil { - return m.MemoryRequest - } - return 0 +func (x *ReplaceContainerMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *CreateContainerMessage) GetStorageRequest() int64 { - if m != nil { - return m.StorageRequest - } - return 0 -} +func (*ReplaceContainerMessage) ProtoMessage() {} -func (m *CreateContainerMessage) GetVolumePlanRequest() map[string]*Volume { - if m != nil { - return m.VolumePlanRequest +func (x *ReplaceContainerMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[57] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } - return nil -} - -type ReplaceContainerMessage struct { - Create *CreateContainerMessage `protobuf:"bytes,1,opt,name=create,proto3" json:"create,omitempty"` - Remove *RemoveContainerMessage `protobuf:"bytes,2,opt,name=remove,proto3" json:"remove,omitempty"` - Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + return mi.MessageOf(x) } -func (m *ReplaceContainerMessage) Reset() { *m = ReplaceContainerMessage{} } -func (m *ReplaceContainerMessage) String() string { return proto.CompactTextString(m) } -func (*ReplaceContainerMessage) ProtoMessage() {} +// Deprecated: Use ReplaceContainerMessage.ProtoReflect.Descriptor instead. func (*ReplaceContainerMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{55} -} - -func (m *ReplaceContainerMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ReplaceContainerMessage.Unmarshal(m, b) -} -func (m *ReplaceContainerMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ReplaceContainerMessage.Marshal(b, m, deterministic) -} -func (m *ReplaceContainerMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_ReplaceContainerMessage.Merge(m, src) -} -func (m *ReplaceContainerMessage) XXX_Size() int { - return xxx_messageInfo_ReplaceContainerMessage.Size(m) -} -func (m *ReplaceContainerMessage) XXX_DiscardUnknown() { - xxx_messageInfo_ReplaceContainerMessage.DiscardUnknown(m) + return file_core_proto_rawDescGZIP(), []int{57} } -var xxx_messageInfo_ReplaceContainerMessage proto.InternalMessageInfo - -func (m *ReplaceContainerMessage) GetCreate() *CreateContainerMessage { - if m != nil { - return m.Create +func (x *ReplaceContainerMessage) GetCreate() *CreateContainerMessage { + if x != nil { + return x.Create } return nil } -func (m *ReplaceContainerMessage) GetRemove() *RemoveContainerMessage { - if m != nil { - return m.Remove +func (x *ReplaceContainerMessage) GetRemove() *RemoveContainerMessage { + if x != nil { + return x.Remove } return nil } -func (m *ReplaceContainerMessage) GetError() string { - if m != nil { - return m.Error +func (x *ReplaceContainerMessage) GetError() string { + if x != nil { + return x.Error } return "" } type CacheImageMessage struct { - Image string `protobuf:"bytes,1,opt,name=image,proto3" json:"image,omitempty"` - Success bool `protobuf:"varint,2,opt,name=success,proto3" json:"success,omitempty"` - Nodename string `protobuf:"bytes,3,opt,name=nodename,proto3" json:"nodename,omitempty"` - Message string `protobuf:"bytes,4,opt,name=message,proto3" json:"message,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CacheImageMessage) Reset() { *m = CacheImageMessage{} } -func (m *CacheImageMessage) String() string { return proto.CompactTextString(m) } -func (*CacheImageMessage) ProtoMessage() {} -func (*CacheImageMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{56} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *CacheImageMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CacheImageMessage.Unmarshal(m, b) -} -func (m *CacheImageMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CacheImageMessage.Marshal(b, m, deterministic) + Image string `protobuf:"bytes,1,opt,name=image,proto3" json:"image,omitempty"` + Success bool `protobuf:"varint,2,opt,name=success,proto3" json:"success,omitempty"` + Nodename string `protobuf:"bytes,3,opt,name=nodename,proto3" json:"nodename,omitempty"` + Message string `protobuf:"bytes,4,opt,name=message,proto3" json:"message,omitempty"` } -func (m *CacheImageMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_CacheImageMessage.Merge(m, src) + +func (x *CacheImageMessage) Reset() { + *x = CacheImageMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[58] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *CacheImageMessage) XXX_Size() int { - return xxx_messageInfo_CacheImageMessage.Size(m) + +func (x *CacheImageMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *CacheImageMessage) XXX_DiscardUnknown() { - xxx_messageInfo_CacheImageMessage.DiscardUnknown(m) + +func (*CacheImageMessage) ProtoMessage() {} + +func (x *CacheImageMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[58] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_CacheImageMessage proto.InternalMessageInfo +// Deprecated: Use CacheImageMessage.ProtoReflect.Descriptor instead. +func (*CacheImageMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{58} +} -func (m *CacheImageMessage) GetImage() string { - if m != nil { - return m.Image +func (x *CacheImageMessage) GetImage() string { + if x != nil { + return x.Image } return "" } -func (m *CacheImageMessage) GetSuccess() bool { - if m != nil { - return m.Success +func (x *CacheImageMessage) GetSuccess() bool { + if x != nil { + return x.Success } return false } -func (m *CacheImageMessage) GetNodename() string { - if m != nil { - return m.Nodename +func (x *CacheImageMessage) GetNodename() string { + if x != nil { + return x.Nodename } return "" } -func (m *CacheImageMessage) GetMessage() string { - if m != nil { - return m.Message +func (x *CacheImageMessage) GetMessage() string { + if x != nil { + return x.Message } return "" } type RemoveImageMessage struct { - Image string `protobuf:"bytes,1,opt,name=image,proto3" json:"image,omitempty"` - Success bool `protobuf:"varint,2,opt,name=success,proto3" json:"success,omitempty"` - Messages []string `protobuf:"bytes,3,rep,name=messages,proto3" json:"messages,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *RemoveImageMessage) Reset() { *m = RemoveImageMessage{} } -func (m *RemoveImageMessage) String() string { return proto.CompactTextString(m) } -func (*RemoveImageMessage) ProtoMessage() {} -func (*RemoveImageMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{57} + Image string `protobuf:"bytes,1,opt,name=image,proto3" json:"image,omitempty"` + Success bool `protobuf:"varint,2,opt,name=success,proto3" json:"success,omitempty"` + Messages []string `protobuf:"bytes,3,rep,name=messages,proto3" json:"messages,omitempty"` } -func (m *RemoveImageMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RemoveImageMessage.Unmarshal(m, b) -} -func (m *RemoveImageMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RemoveImageMessage.Marshal(b, m, deterministic) -} -func (m *RemoveImageMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_RemoveImageMessage.Merge(m, src) +func (x *RemoveImageMessage) Reset() { + *x = RemoveImageMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[59] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *RemoveImageMessage) XXX_Size() int { - return xxx_messageInfo_RemoveImageMessage.Size(m) + +func (x *RemoveImageMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *RemoveImageMessage) XXX_DiscardUnknown() { - xxx_messageInfo_RemoveImageMessage.DiscardUnknown(m) + +func (*RemoveImageMessage) ProtoMessage() {} + +func (x *RemoveImageMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[59] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_RemoveImageMessage proto.InternalMessageInfo +// Deprecated: Use RemoveImageMessage.ProtoReflect.Descriptor instead. +func (*RemoveImageMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{59} +} -func (m *RemoveImageMessage) GetImage() string { - if m != nil { - return m.Image +func (x *RemoveImageMessage) GetImage() string { + if x != nil { + return x.Image } return "" } -func (m *RemoveImageMessage) GetSuccess() bool { - if m != nil { - return m.Success +func (x *RemoveImageMessage) GetSuccess() bool { + if x != nil { + return x.Success } return false } -func (m *RemoveImageMessage) GetMessages() []string { - if m != nil { - return m.Messages +func (x *RemoveImageMessage) GetMessages() []string { + if x != nil { + return x.Messages } return nil } type RemoveContainerMessage struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Success bool `protobuf:"varint,2,opt,name=success,proto3" json:"success,omitempty"` - Hook string `protobuf:"bytes,3,opt,name=hook,proto3" json:"hook,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *RemoveContainerMessage) Reset() { *m = RemoveContainerMessage{} } -func (m *RemoveContainerMessage) String() string { return proto.CompactTextString(m) } -func (*RemoveContainerMessage) ProtoMessage() {} -func (*RemoveContainerMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{58} + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Success bool `protobuf:"varint,2,opt,name=success,proto3" json:"success,omitempty"` + Hook string `protobuf:"bytes,3,opt,name=hook,proto3" json:"hook,omitempty"` } -func (m *RemoveContainerMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RemoveContainerMessage.Unmarshal(m, b) -} -func (m *RemoveContainerMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RemoveContainerMessage.Marshal(b, m, deterministic) -} -func (m *RemoveContainerMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_RemoveContainerMessage.Merge(m, src) +func (x *RemoveContainerMessage) Reset() { + *x = RemoveContainerMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *RemoveContainerMessage) XXX_Size() int { - return xxx_messageInfo_RemoveContainerMessage.Size(m) + +func (x *RemoveContainerMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *RemoveContainerMessage) XXX_DiscardUnknown() { - xxx_messageInfo_RemoveContainerMessage.DiscardUnknown(m) + +func (*RemoveContainerMessage) ProtoMessage() {} + +func (x *RemoveContainerMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[60] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_RemoveContainerMessage proto.InternalMessageInfo +// Deprecated: Use RemoveContainerMessage.ProtoReflect.Descriptor instead. +func (*RemoveContainerMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{60} +} -func (m *RemoveContainerMessage) GetId() string { - if m != nil { - return m.Id +func (x *RemoveContainerMessage) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *RemoveContainerMessage) GetSuccess() bool { - if m != nil { - return m.Success +func (x *RemoveContainerMessage) GetSuccess() bool { + if x != nil { + return x.Success } return false } -func (m *RemoveContainerMessage) GetHook() string { - if m != nil { - return m.Hook +func (x *RemoveContainerMessage) GetHook() string { + if x != nil { + return x.Hook } return "" } type DissociateContainerMessage struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *DissociateContainerMessage) Reset() { *m = DissociateContainerMessage{} } -func (m *DissociateContainerMessage) String() string { return proto.CompactTextString(m) } -func (*DissociateContainerMessage) ProtoMessage() {} -func (*DissociateContainerMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{59} + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` } -func (m *DissociateContainerMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_DissociateContainerMessage.Unmarshal(m, b) -} -func (m *DissociateContainerMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_DissociateContainerMessage.Marshal(b, m, deterministic) -} -func (m *DissociateContainerMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_DissociateContainerMessage.Merge(m, src) +func (x *DissociateContainerMessage) Reset() { + *x = DissociateContainerMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[61] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *DissociateContainerMessage) XXX_Size() int { - return xxx_messageInfo_DissociateContainerMessage.Size(m) + +func (x *DissociateContainerMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *DissociateContainerMessage) XXX_DiscardUnknown() { - xxx_messageInfo_DissociateContainerMessage.DiscardUnknown(m) + +func (*DissociateContainerMessage) ProtoMessage() {} + +func (x *DissociateContainerMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[61] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_DissociateContainerMessage proto.InternalMessageInfo +// Deprecated: Use DissociateContainerMessage.ProtoReflect.Descriptor instead. +func (*DissociateContainerMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{61} +} -func (m *DissociateContainerMessage) GetId() string { - if m != nil { - return m.Id +func (x *DissociateContainerMessage) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *DissociateContainerMessage) GetError() string { - if m != nil { - return m.Error +func (x *DissociateContainerMessage) GetError() string { + if x != nil { + return x.Error } return "" } type ReallocResourceMessage struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ReallocResourceMessage) Reset() { *m = ReallocResourceMessage{} } -func (m *ReallocResourceMessage) String() string { return proto.CompactTextString(m) } -func (*ReallocResourceMessage) ProtoMessage() {} -func (*ReallocResourceMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{60} + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` } -func (m *ReallocResourceMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ReallocResourceMessage.Unmarshal(m, b) -} -func (m *ReallocResourceMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ReallocResourceMessage.Marshal(b, m, deterministic) -} -func (m *ReallocResourceMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_ReallocResourceMessage.Merge(m, src) -} -func (m *ReallocResourceMessage) XXX_Size() int { - return xxx_messageInfo_ReallocResourceMessage.Size(m) +func (x *ReallocResourceMessage) Reset() { + *x = ReallocResourceMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ReallocResourceMessage) XXX_DiscardUnknown() { - xxx_messageInfo_ReallocResourceMessage.DiscardUnknown(m) + +func (x *ReallocResourceMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -var xxx_messageInfo_ReallocResourceMessage proto.InternalMessageInfo +func (*ReallocResourceMessage) ProtoMessage() {} -func (m *ReallocResourceMessage) GetId() string { - if m != nil { - return m.Id +func (x *ReallocResourceMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[62] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } - return "" + return mi.MessageOf(x) +} + +// Deprecated: Use ReallocResourceMessage.ProtoReflect.Descriptor instead. +func (*ReallocResourceMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{62} } -func (m *ReallocResourceMessage) GetError() string { - if m != nil { - return m.Error +func (x *ReallocResourceMessage) GetError() string { + if x != nil { + return x.Error } return "" } type CopyMessage struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Status string `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - Name string `protobuf:"bytes,3,opt,name=name,proto3" json:"name,omitempty"` - Path string `protobuf:"bytes,4,opt,name=path,proto3" json:"path,omitempty"` - Error string `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"` - Data []byte `protobuf:"bytes,6,opt,name=data,proto3" json:"data,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CopyMessage) Reset() { *m = CopyMessage{} } -func (m *CopyMessage) String() string { return proto.CompactTextString(m) } -func (*CopyMessage) ProtoMessage() {} -func (*CopyMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{61} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *CopyMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CopyMessage.Unmarshal(m, b) + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Status string `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` + Name string `protobuf:"bytes,3,opt,name=name,proto3" json:"name,omitempty"` + Path string `protobuf:"bytes,4,opt,name=path,proto3" json:"path,omitempty"` + Error string `protobuf:"bytes,5,opt,name=error,proto3" json:"error,omitempty"` + Data []byte `protobuf:"bytes,6,opt,name=data,proto3" json:"data,omitempty"` } -func (m *CopyMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CopyMessage.Marshal(b, m, deterministic) -} -func (m *CopyMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_CopyMessage.Merge(m, src) + +func (x *CopyMessage) Reset() { + *x = CopyMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *CopyMessage) XXX_Size() int { - return xxx_messageInfo_CopyMessage.Size(m) + +func (x *CopyMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *CopyMessage) XXX_DiscardUnknown() { - xxx_messageInfo_CopyMessage.DiscardUnknown(m) + +func (*CopyMessage) ProtoMessage() {} + +func (x *CopyMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[63] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_CopyMessage proto.InternalMessageInfo +// Deprecated: Use CopyMessage.ProtoReflect.Descriptor instead. +func (*CopyMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{63} +} -func (m *CopyMessage) GetId() string { - if m != nil { - return m.Id +func (x *CopyMessage) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *CopyMessage) GetStatus() string { - if m != nil { - return m.Status +func (x *CopyMessage) GetStatus() string { + if x != nil { + return x.Status } return "" } -func (m *CopyMessage) GetName() string { - if m != nil { - return m.Name +func (x *CopyMessage) GetName() string { + if x != nil { + return x.Name } return "" } -func (m *CopyMessage) GetPath() string { - if m != nil { - return m.Path +func (x *CopyMessage) GetPath() string { + if x != nil { + return x.Path } return "" } -func (m *CopyMessage) GetError() string { - if m != nil { - return m.Error +func (x *CopyMessage) GetError() string { + if x != nil { + return x.Error } return "" } -func (m *CopyMessage) GetData() []byte { - if m != nil { - return m.Data +func (x *CopyMessage) GetData() []byte { + if x != nil { + return x.Data } return nil } type SendMessage struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` - Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *SendMessage) Reset() { *m = SendMessage{} } -func (m *SendMessage) String() string { return proto.CompactTextString(m) } -func (*SendMessage) ProtoMessage() {} -func (*SendMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{62} + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` + Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` } -func (m *SendMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SendMessage.Unmarshal(m, b) -} -func (m *SendMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SendMessage.Marshal(b, m, deterministic) -} -func (m *SendMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_SendMessage.Merge(m, src) +func (x *SendMessage) Reset() { + *x = SendMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *SendMessage) XXX_Size() int { - return xxx_messageInfo_SendMessage.Size(m) + +func (x *SendMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *SendMessage) XXX_DiscardUnknown() { - xxx_messageInfo_SendMessage.DiscardUnknown(m) + +func (*SendMessage) ProtoMessage() {} + +func (x *SendMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[64] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_SendMessage proto.InternalMessageInfo +// Deprecated: Use SendMessage.ProtoReflect.Descriptor instead. +func (*SendMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{64} +} -func (m *SendMessage) GetId() string { - if m != nil { - return m.Id +func (x *SendMessage) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *SendMessage) GetPath() string { - if m != nil { - return m.Path +func (x *SendMessage) GetPath() string { + if x != nil { + return x.Path } return "" } -func (m *SendMessage) GetError() string { - if m != nil { - return m.Error +func (x *SendMessage) GetError() string { + if x != nil { + return x.Error } return "" } type AttachContainerMessage struct { - ContainerId string `protobuf:"bytes,1,opt,name=container_id,json=containerId,proto3" json:"container_id,omitempty"` - Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *AttachContainerMessage) Reset() { *m = AttachContainerMessage{} } -func (m *AttachContainerMessage) String() string { return proto.CompactTextString(m) } -func (*AttachContainerMessage) ProtoMessage() {} -func (*AttachContainerMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{63} + ContainerId string `protobuf:"bytes,1,opt,name=container_id,json=containerId,proto3" json:"container_id,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` } -func (m *AttachContainerMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_AttachContainerMessage.Unmarshal(m, b) -} -func (m *AttachContainerMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_AttachContainerMessage.Marshal(b, m, deterministic) -} -func (m *AttachContainerMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_AttachContainerMessage.Merge(m, src) +func (x *AttachContainerMessage) Reset() { + *x = AttachContainerMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *AttachContainerMessage) XXX_Size() int { - return xxx_messageInfo_AttachContainerMessage.Size(m) + +func (x *AttachContainerMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *AttachContainerMessage) XXX_DiscardUnknown() { - xxx_messageInfo_AttachContainerMessage.DiscardUnknown(m) + +func (*AttachContainerMessage) ProtoMessage() {} + +func (x *AttachContainerMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[65] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_AttachContainerMessage proto.InternalMessageInfo +// Deprecated: Use AttachContainerMessage.ProtoReflect.Descriptor instead. +func (*AttachContainerMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{65} +} -func (m *AttachContainerMessage) GetContainerId() string { - if m != nil { - return m.ContainerId +func (x *AttachContainerMessage) GetContainerId() string { + if x != nil { + return x.ContainerId } return "" } -func (m *AttachContainerMessage) GetData() []byte { - if m != nil { - return m.Data +func (x *AttachContainerMessage) GetData() []byte { + if x != nil { + return x.Data } return nil } type RunAndWaitOptions struct { - DeployOptions *DeployOptions `protobuf:"bytes,1,opt,name=deploy_options,json=deployOptions,proto3" json:"deploy_options,omitempty"` - Cmd []byte `protobuf:"bytes,2,opt,name=cmd,proto3" json:"cmd,omitempty"` - Async bool `protobuf:"varint,3,opt,name=async,proto3" json:"async,omitempty"` - AsyncTimeout int32 `protobuf:"varint,4,opt,name=async_timeout,json=asyncTimeout,proto3" json:"async_timeout,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *RunAndWaitOptions) Reset() { *m = RunAndWaitOptions{} } -func (m *RunAndWaitOptions) String() string { return proto.CompactTextString(m) } -func (*RunAndWaitOptions) ProtoMessage() {} -func (*RunAndWaitOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{64} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *RunAndWaitOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RunAndWaitOptions.Unmarshal(m, b) -} -func (m *RunAndWaitOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RunAndWaitOptions.Marshal(b, m, deterministic) + DeployOptions *DeployOptions `protobuf:"bytes,1,opt,name=deploy_options,json=deployOptions,proto3" json:"deploy_options,omitempty"` + Cmd []byte `protobuf:"bytes,2,opt,name=cmd,proto3" json:"cmd,omitempty"` + Async bool `protobuf:"varint,3,opt,name=async,proto3" json:"async,omitempty"` + AsyncTimeout int32 `protobuf:"varint,4,opt,name=async_timeout,json=asyncTimeout,proto3" json:"async_timeout,omitempty"` } -func (m *RunAndWaitOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_RunAndWaitOptions.Merge(m, src) + +func (x *RunAndWaitOptions) Reset() { + *x = RunAndWaitOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[66] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *RunAndWaitOptions) XXX_Size() int { - return xxx_messageInfo_RunAndWaitOptions.Size(m) + +func (x *RunAndWaitOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *RunAndWaitOptions) XXX_DiscardUnknown() { - xxx_messageInfo_RunAndWaitOptions.DiscardUnknown(m) + +func (*RunAndWaitOptions) ProtoMessage() {} + +func (x *RunAndWaitOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[66] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_RunAndWaitOptions proto.InternalMessageInfo +// Deprecated: Use RunAndWaitOptions.ProtoReflect.Descriptor instead. +func (*RunAndWaitOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{66} +} -func (m *RunAndWaitOptions) GetDeployOptions() *DeployOptions { - if m != nil { - return m.DeployOptions +func (x *RunAndWaitOptions) GetDeployOptions() *DeployOptions { + if x != nil { + return x.DeployOptions } return nil } -func (m *RunAndWaitOptions) GetCmd() []byte { - if m != nil { - return m.Cmd +func (x *RunAndWaitOptions) GetCmd() []byte { + if x != nil { + return x.Cmd } return nil } -func (m *RunAndWaitOptions) GetAsync() bool { - if m != nil { - return m.Async +func (x *RunAndWaitOptions) GetAsync() bool { + if x != nil { + return x.Async } return false } -func (m *RunAndWaitOptions) GetAsyncTimeout() int32 { - if m != nil { - return m.AsyncTimeout +func (x *RunAndWaitOptions) GetAsyncTimeout() int32 { + if x != nil { + return x.AsyncTimeout } return 0 } type ControlContainerOptions struct { - Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` - Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` - Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ControlContainerOptions) Reset() { *m = ControlContainerOptions{} } -func (m *ControlContainerOptions) String() string { return proto.CompactTextString(m) } -func (*ControlContainerOptions) ProtoMessage() {} -func (*ControlContainerOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{65} + Ids []string `protobuf:"bytes,1,rep,name=ids,proto3" json:"ids,omitempty"` + Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"` } -func (m *ControlContainerOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ControlContainerOptions.Unmarshal(m, b) -} -func (m *ControlContainerOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ControlContainerOptions.Marshal(b, m, deterministic) -} -func (m *ControlContainerOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_ControlContainerOptions.Merge(m, src) +func (x *ControlContainerOptions) Reset() { + *x = ControlContainerOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[67] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ControlContainerOptions) XXX_Size() int { - return xxx_messageInfo_ControlContainerOptions.Size(m) + +func (x *ControlContainerOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ControlContainerOptions) XXX_DiscardUnknown() { - xxx_messageInfo_ControlContainerOptions.DiscardUnknown(m) + +func (*ControlContainerOptions) ProtoMessage() {} + +func (x *ControlContainerOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[67] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ControlContainerOptions proto.InternalMessageInfo +// Deprecated: Use ControlContainerOptions.ProtoReflect.Descriptor instead. +func (*ControlContainerOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{67} +} -func (m *ControlContainerOptions) GetIds() []string { - if m != nil { - return m.Ids +func (x *ControlContainerOptions) GetIds() []string { + if x != nil { + return x.Ids } return nil } -func (m *ControlContainerOptions) GetType() string { - if m != nil { - return m.Type +func (x *ControlContainerOptions) GetType() string { + if x != nil { + return x.Type } return "" } -func (m *ControlContainerOptions) GetForce() bool { - if m != nil { - return m.Force +func (x *ControlContainerOptions) GetForce() bool { + if x != nil { + return x.Force } return false } type ControlContainerMessage struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` - Hook []byte `protobuf:"bytes,3,opt,name=hook,proto3" json:"hook,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ControlContainerMessage) Reset() { *m = ControlContainerMessage{} } -func (m *ControlContainerMessage) String() string { return proto.CompactTextString(m) } -func (*ControlContainerMessage) ProtoMessage() {} -func (*ControlContainerMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{66} + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + Hook []byte `protobuf:"bytes,3,opt,name=hook,proto3" json:"hook,omitempty"` } -func (m *ControlContainerMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ControlContainerMessage.Unmarshal(m, b) -} -func (m *ControlContainerMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ControlContainerMessage.Marshal(b, m, deterministic) -} -func (m *ControlContainerMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_ControlContainerMessage.Merge(m, src) +func (x *ControlContainerMessage) Reset() { + *x = ControlContainerMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[68] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ControlContainerMessage) XXX_Size() int { - return xxx_messageInfo_ControlContainerMessage.Size(m) + +func (x *ControlContainerMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ControlContainerMessage) XXX_DiscardUnknown() { - xxx_messageInfo_ControlContainerMessage.DiscardUnknown(m) + +func (*ControlContainerMessage) ProtoMessage() {} + +func (x *ControlContainerMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[68] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ControlContainerMessage proto.InternalMessageInfo +// Deprecated: Use ControlContainerMessage.ProtoReflect.Descriptor instead. +func (*ControlContainerMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{68} +} -func (m *ControlContainerMessage) GetId() string { - if m != nil { - return m.Id +func (x *ControlContainerMessage) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *ControlContainerMessage) GetError() string { - if m != nil { - return m.Error +func (x *ControlContainerMessage) GetError() string { + if x != nil { + return x.Error } return "" } -func (m *ControlContainerMessage) GetHook() []byte { - if m != nil { - return m.Hook +func (x *ControlContainerMessage) GetHook() []byte { + if x != nil { + return x.Hook } return nil } type LogStreamOptions struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Tail string `protobuf:"bytes,2,opt,name=tail,proto3" json:"tail,omitempty"` - Since string `protobuf:"bytes,3,opt,name=since,proto3" json:"since,omitempty"` - Until string `protobuf:"bytes,4,opt,name=until,proto3" json:"until,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *LogStreamOptions) Reset() { *m = LogStreamOptions{} } -func (m *LogStreamOptions) String() string { return proto.CompactTextString(m) } -func (*LogStreamOptions) ProtoMessage() {} -func (*LogStreamOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{67} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *LogStreamOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_LogStreamOptions.Unmarshal(m, b) + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Tail string `protobuf:"bytes,2,opt,name=tail,proto3" json:"tail,omitempty"` + Since string `protobuf:"bytes,3,opt,name=since,proto3" json:"since,omitempty"` + Until string `protobuf:"bytes,4,opt,name=until,proto3" json:"until,omitempty"` } -func (m *LogStreamOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_LogStreamOptions.Marshal(b, m, deterministic) -} -func (m *LogStreamOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_LogStreamOptions.Merge(m, src) + +func (x *LogStreamOptions) Reset() { + *x = LogStreamOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[69] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *LogStreamOptions) XXX_Size() int { - return xxx_messageInfo_LogStreamOptions.Size(m) + +func (x *LogStreamOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *LogStreamOptions) XXX_DiscardUnknown() { - xxx_messageInfo_LogStreamOptions.DiscardUnknown(m) + +func (*LogStreamOptions) ProtoMessage() {} + +func (x *LogStreamOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[69] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_LogStreamOptions proto.InternalMessageInfo +// Deprecated: Use LogStreamOptions.ProtoReflect.Descriptor instead. +func (*LogStreamOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{69} +} -func (m *LogStreamOptions) GetId() string { - if m != nil { - return m.Id +func (x *LogStreamOptions) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *LogStreamOptions) GetTail() string { - if m != nil { - return m.Tail +func (x *LogStreamOptions) GetTail() string { + if x != nil { + return x.Tail } return "" } -func (m *LogStreamOptions) GetSince() string { - if m != nil { - return m.Since +func (x *LogStreamOptions) GetSince() string { + if x != nil { + return x.Since } return "" } -func (m *LogStreamOptions) GetUntil() string { - if m != nil { - return m.Until +func (x *LogStreamOptions) GetUntil() string { + if x != nil { + return x.Until } return "" } type LogStreamMessage struct { - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` - Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *LogStreamMessage) Reset() { *m = LogStreamMessage{} } -func (m *LogStreamMessage) String() string { return proto.CompactTextString(m) } -func (*LogStreamMessage) ProtoMessage() {} -func (*LogStreamMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{68} + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"` } -func (m *LogStreamMessage) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_LogStreamMessage.Unmarshal(m, b) -} -func (m *LogStreamMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_LogStreamMessage.Marshal(b, m, deterministic) -} -func (m *LogStreamMessage) XXX_Merge(src proto.Message) { - xxx_messageInfo_LogStreamMessage.Merge(m, src) +func (x *LogStreamMessage) Reset() { + *x = LogStreamMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[70] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *LogStreamMessage) XXX_Size() int { - return xxx_messageInfo_LogStreamMessage.Size(m) + +func (x *LogStreamMessage) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *LogStreamMessage) XXX_DiscardUnknown() { - xxx_messageInfo_LogStreamMessage.DiscardUnknown(m) + +func (*LogStreamMessage) ProtoMessage() {} + +func (x *LogStreamMessage) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[70] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_LogStreamMessage proto.InternalMessageInfo +// Deprecated: Use LogStreamMessage.ProtoReflect.Descriptor instead. +func (*LogStreamMessage) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{70} +} -func (m *LogStreamMessage) GetId() string { - if m != nil { - return m.Id +func (x *LogStreamMessage) GetId() string { + if x != nil { + return x.Id } return "" } -func (m *LogStreamMessage) GetError() string { - if m != nil { - return m.Error +func (x *LogStreamMessage) GetError() string { + if x != nil { + return x.Error } return "" } -func (m *LogStreamMessage) GetData() []byte { - if m != nil { - return m.Data +func (x *LogStreamMessage) GetData() []byte { + if x != nil { + return x.Data } return nil } type ExecuteContainerOptions struct { - ContainerId string `protobuf:"bytes,1,opt,name=container_id,json=containerId,proto3" json:"container_id,omitempty"` - Commands []string `protobuf:"bytes,2,rep,name=commands,proto3" json:"commands,omitempty"` - Envs []string `protobuf:"bytes,3,rep,name=envs,proto3" json:"envs,omitempty"` - Workdir string `protobuf:"bytes,4,opt,name=workdir,proto3" json:"workdir,omitempty"` - OpenStdin bool `protobuf:"varint,5,opt,name=open_stdin,json=openStdin,proto3" json:"open_stdin,omitempty"` - ReplCmd []byte `protobuf:"bytes,6,opt,name=repl_cmd,json=replCmd,proto3" json:"repl_cmd,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ExecuteContainerOptions) Reset() { *m = ExecuteContainerOptions{} } -func (m *ExecuteContainerOptions) String() string { return proto.CompactTextString(m) } -func (*ExecuteContainerOptions) ProtoMessage() {} -func (*ExecuteContainerOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_f7e43720d1edc0fe, []int{69} -} + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (m *ExecuteContainerOptions) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ExecuteContainerOptions.Unmarshal(m, b) + ContainerId string `protobuf:"bytes,1,opt,name=container_id,json=containerId,proto3" json:"container_id,omitempty"` + Commands []string `protobuf:"bytes,2,rep,name=commands,proto3" json:"commands,omitempty"` + Envs []string `protobuf:"bytes,3,rep,name=envs,proto3" json:"envs,omitempty"` + Workdir string `protobuf:"bytes,4,opt,name=workdir,proto3" json:"workdir,omitempty"` + OpenStdin bool `protobuf:"varint,5,opt,name=open_stdin,json=openStdin,proto3" json:"open_stdin,omitempty"` + ReplCmd []byte `protobuf:"bytes,6,opt,name=repl_cmd,json=replCmd,proto3" json:"repl_cmd,omitempty"` } -func (m *ExecuteContainerOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ExecuteContainerOptions.Marshal(b, m, deterministic) -} -func (m *ExecuteContainerOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_ExecuteContainerOptions.Merge(m, src) + +func (x *ExecuteContainerOptions) Reset() { + *x = ExecuteContainerOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_core_proto_msgTypes[71] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } } -func (m *ExecuteContainerOptions) XXX_Size() int { - return xxx_messageInfo_ExecuteContainerOptions.Size(m) + +func (x *ExecuteContainerOptions) String() string { + return protoimpl.X.MessageStringOf(x) } -func (m *ExecuteContainerOptions) XXX_DiscardUnknown() { - xxx_messageInfo_ExecuteContainerOptions.DiscardUnknown(m) + +func (*ExecuteContainerOptions) ProtoMessage() {} + +func (x *ExecuteContainerOptions) ProtoReflect() protoreflect.Message { + mi := &file_core_proto_msgTypes[71] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -var xxx_messageInfo_ExecuteContainerOptions proto.InternalMessageInfo +// Deprecated: Use ExecuteContainerOptions.ProtoReflect.Descriptor instead. +func (*ExecuteContainerOptions) Descriptor() ([]byte, []int) { + return file_core_proto_rawDescGZIP(), []int{71} +} -func (m *ExecuteContainerOptions) GetContainerId() string { - if m != nil { - return m.ContainerId +func (x *ExecuteContainerOptions) GetContainerId() string { + if x != nil { + return x.ContainerId } return "" } -func (m *ExecuteContainerOptions) GetCommands() []string { - if m != nil { - return m.Commands +func (x *ExecuteContainerOptions) GetCommands() []string { + if x != nil { + return x.Commands } return nil } -func (m *ExecuteContainerOptions) GetEnvs() []string { - if m != nil { - return m.Envs +func (x *ExecuteContainerOptions) GetEnvs() []string { + if x != nil { + return x.Envs } return nil } -func (m *ExecuteContainerOptions) GetWorkdir() string { - if m != nil { - return m.Workdir +func (x *ExecuteContainerOptions) GetWorkdir() string { + if x != nil { + return x.Workdir } return "" } -func (m *ExecuteContainerOptions) GetOpenStdin() bool { - if m != nil { - return m.OpenStdin +func (x *ExecuteContainerOptions) GetOpenStdin() bool { + if x != nil { + return x.OpenStdin } return false } -func (m *ExecuteContainerOptions) GetReplCmd() []byte { - if m != nil { - return m.ReplCmd +func (x *ExecuteContainerOptions) GetReplCmd() []byte { + if x != nil { + return x.ReplCmd } return nil } -func init() { - proto.RegisterEnum("pb.TriOpt", TriOpt_name, TriOpt_value) - proto.RegisterEnum("pb.BuildImageOptions_BuildMethod", BuildImageOptions_BuildMethod_name, BuildImageOptions_BuildMethod_value) - proto.RegisterEnum("pb.DeployOptions_Strategy", DeployOptions_Strategy_name, DeployOptions_Strategy_value) - proto.RegisterType((*Empty)(nil), "pb.Empty") - proto.RegisterType((*CoreInfo)(nil), "pb.CoreInfo") - proto.RegisterType((*ServiceStatus)(nil), "pb.ServiceStatus") - proto.RegisterType((*ListContainersOptions)(nil), "pb.ListContainersOptions") - proto.RegisterMapType((map[string]string)(nil), "pb.ListContainersOptions.LabelsEntry") - proto.RegisterType((*Pod)(nil), "pb.Pod") - proto.RegisterType((*Pods)(nil), "pb.Pods") - proto.RegisterType((*PodResource)(nil), "pb.PodResource") - proto.RegisterMapType((map[string]float64)(nil), "pb.PodResource.CpuPercentsEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.PodResource.DetailsEntry") - proto.RegisterMapType((map[string]float64)(nil), "pb.PodResource.MemoryPercentsEntry") - proto.RegisterMapType((map[string]float64)(nil), "pb.PodResource.StoragePercentsEntry") - proto.RegisterMapType((map[string]bool)(nil), "pb.PodResource.VerificationsEntry") - proto.RegisterMapType((map[string]float64)(nil), "pb.PodResource.VolumePercentsEntry") - proto.RegisterType((*NodeResource)(nil), "pb.NodeResource") - proto.RegisterType((*ListNetworkOptions)(nil), "pb.ListNetworkOptions") - proto.RegisterType((*ConnectNetworkOptions)(nil), "pb.ConnectNetworkOptions") - proto.RegisterType((*DisconnectNetworkOptions)(nil), "pb.DisconnectNetworkOptions") - proto.RegisterType((*Network)(nil), "pb.Network") - proto.RegisterType((*Networks)(nil), "pb.Networks") - proto.RegisterType((*Node)(nil), "pb.Node") - proto.RegisterMapType((map[string]int32)(nil), "pb.Node.CpuEntry") - proto.RegisterMapType((map[string]int32)(nil), "pb.Node.InitCpuEntry") - proto.RegisterMapType((map[string]int64)(nil), "pb.Node.InitVolumeEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.Node.LabelsEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.Node.NumaEntry") - proto.RegisterMapType((map[string]int64)(nil), "pb.Node.NumaMemoryEntry") - proto.RegisterMapType((map[string]int64)(nil), "pb.Node.VolumeEntry") - proto.RegisterType((*Nodes)(nil), "pb.Nodes") - proto.RegisterType((*NodeAvailable)(nil), "pb.NodeAvailable") - proto.RegisterType((*SetNodeOptions)(nil), "pb.SetNodeOptions") - proto.RegisterMapType((map[string]int32)(nil), "pb.SetNodeOptions.DeltaCpuEntry") - proto.RegisterMapType((map[string]int64)(nil), "pb.SetNodeOptions.DeltaNumaMemoryEntry") - proto.RegisterMapType((map[string]int64)(nil), "pb.SetNodeOptions.DeltaVolumeEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.SetNodeOptions.LabelsEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.SetNodeOptions.NumaEntry") - proto.RegisterType((*Container)(nil), "pb.Container") - proto.RegisterMapType((map[string]int32)(nil), "pb.Container.CpuEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.Container.LabelsEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.Container.PublishEntry") - proto.RegisterMapType((map[string]*Volume)(nil), "pb.Container.VolumePlanEntry") - proto.RegisterMapType((map[string]*Volume)(nil), "pb.Container.VolumePlanRequestEntry") - proto.RegisterType((*ContainerStatus)(nil), "pb.ContainerStatus") - proto.RegisterMapType((map[string]string)(nil), "pb.ContainerStatus.NetworksEntry") - proto.RegisterType((*ContainersStatus)(nil), "pb.ContainersStatus") - proto.RegisterType((*ContainerStatusStreamMessage)(nil), "pb.ContainerStatusStreamMessage") - proto.RegisterType((*SetContainersStatusOptions)(nil), "pb.SetContainersStatusOptions") - proto.RegisterType((*ContainerStatusStreamOptions)(nil), "pb.ContainerStatusStreamOptions") - proto.RegisterMapType((map[string]string)(nil), "pb.ContainerStatusStreamOptions.LabelsEntry") - proto.RegisterType((*Containers)(nil), "pb.Containers") - proto.RegisterType((*ContainerID)(nil), "pb.ContainerID") - proto.RegisterType((*ContainerIDs)(nil), "pb.ContainerIDs") - proto.RegisterType((*RemoveContainerOptions)(nil), "pb.RemoveContainerOptions") - proto.RegisterType((*DissociateContainerOptions)(nil), "pb.DissociateContainerOptions") - proto.RegisterType((*ReallocOptions)(nil), "pb.ReallocOptions") - proto.RegisterType((*AddPodOptions)(nil), "pb.AddPodOptions") - proto.RegisterType((*RemovePodOptions)(nil), "pb.RemovePodOptions") - proto.RegisterType((*GetPodOptions)(nil), "pb.GetPodOptions") - proto.RegisterType((*AddNodeOptions)(nil), "pb.AddNodeOptions") - proto.RegisterMapType((map[string]string)(nil), "pb.AddNodeOptions.LabelsEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.AddNodeOptions.NumaEntry") - proto.RegisterMapType((map[string]int64)(nil), "pb.AddNodeOptions.NumaMemoryEntry") - proto.RegisterMapType((map[string]int64)(nil), "pb.AddNodeOptions.VolumeMapEntry") - proto.RegisterType((*RemoveNodeOptions)(nil), "pb.RemoveNodeOptions") - proto.RegisterType((*GetNodeOptions)(nil), "pb.GetNodeOptions") - proto.RegisterMapType((map[string]string)(nil), "pb.GetNodeOptions.LabelsEntry") - proto.RegisterType((*GetNodeResourceOptions)(nil), "pb.GetNodeResourceOptions") - proto.RegisterType((*ListNodesOptions)(nil), "pb.ListNodesOptions") - proto.RegisterMapType((map[string]string)(nil), "pb.ListNodesOptions.LabelsEntry") - proto.RegisterType((*Build)(nil), "pb.Build") - proto.RegisterMapType((map[string]string)(nil), "pb.Build.ArgsEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.Build.ArtifactsEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.Build.CacheEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.Build.EnvsEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.Build.LabelsEntry") - proto.RegisterType((*Builds)(nil), "pb.Builds") - proto.RegisterMapType((map[string]*Build)(nil), "pb.Builds.BuildsEntry") - proto.RegisterType((*BuildImageOptions)(nil), "pb.BuildImageOptions") - proto.RegisterType((*HookOptions)(nil), "pb.HookOptions") - proto.RegisterType((*HealthCheckOptions)(nil), "pb.HealthCheckOptions") - proto.RegisterType((*LogOptions)(nil), "pb.LogOptions") - proto.RegisterMapType((map[string]string)(nil), "pb.LogOptions.ConfigEntry") - proto.RegisterType((*EntrypointOptions)(nil), "pb.EntrypointOptions") - proto.RegisterMapType((map[string]string)(nil), "pb.EntrypointOptions.SysctlsEntry") - proto.RegisterType((*DeployOptions)(nil), "pb.DeployOptions") - proto.RegisterMapType((map[string][]byte)(nil), "pb.DeployOptions.DataEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.DeployOptions.LabelsEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.DeployOptions.NetworksEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.DeployOptions.NodelabelsEntry") - proto.RegisterType((*ReplaceOptions)(nil), "pb.ReplaceOptions") - proto.RegisterMapType((map[string]string)(nil), "pb.ReplaceOptions.CopyEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.ReplaceOptions.FilterLabelsEntry") - proto.RegisterType((*CacheImageOptions)(nil), "pb.CacheImageOptions") - proto.RegisterType((*RemoveImageOptions)(nil), "pb.RemoveImageOptions") - proto.RegisterType((*CopyPaths)(nil), "pb.CopyPaths") - proto.RegisterType((*CopyOptions)(nil), "pb.CopyOptions") - proto.RegisterMapType((map[string]*CopyPaths)(nil), "pb.CopyOptions.TargetsEntry") - proto.RegisterType((*SendOptions)(nil), "pb.SendOptions") - proto.RegisterMapType((map[string][]byte)(nil), "pb.SendOptions.DataEntry") - proto.RegisterType((*ErrorDetail)(nil), "pb.ErrorDetail") - proto.RegisterType((*BuildImageMessage)(nil), "pb.BuildImageMessage") - proto.RegisterType((*Volume)(nil), "pb.Volume") - proto.RegisterMapType((map[string]int64)(nil), "pb.Volume.VolumeEntry") - proto.RegisterType((*CreateContainerMessage)(nil), "pb.CreateContainerMessage") - proto.RegisterMapType((map[string]int32)(nil), "pb.CreateContainerMessage.CpuEntry") - proto.RegisterMapType((map[string]string)(nil), "pb.CreateContainerMessage.PublishEntry") - proto.RegisterMapType((map[string]*Volume)(nil), "pb.CreateContainerMessage.VolumePlanEntry") - proto.RegisterMapType((map[string]*Volume)(nil), "pb.CreateContainerMessage.VolumePlanRequestEntry") - proto.RegisterType((*ReplaceContainerMessage)(nil), "pb.ReplaceContainerMessage") - proto.RegisterType((*CacheImageMessage)(nil), "pb.CacheImageMessage") - proto.RegisterType((*RemoveImageMessage)(nil), "pb.RemoveImageMessage") - proto.RegisterType((*RemoveContainerMessage)(nil), "pb.RemoveContainerMessage") - proto.RegisterType((*DissociateContainerMessage)(nil), "pb.DissociateContainerMessage") - proto.RegisterType((*ReallocResourceMessage)(nil), "pb.ReallocResourceMessage") - proto.RegisterType((*CopyMessage)(nil), "pb.CopyMessage") - proto.RegisterType((*SendMessage)(nil), "pb.SendMessage") - proto.RegisterType((*AttachContainerMessage)(nil), "pb.AttachContainerMessage") - proto.RegisterType((*RunAndWaitOptions)(nil), "pb.RunAndWaitOptions") - proto.RegisterType((*ControlContainerOptions)(nil), "pb.ControlContainerOptions") - proto.RegisterType((*ControlContainerMessage)(nil), "pb.ControlContainerMessage") - proto.RegisterType((*LogStreamOptions)(nil), "pb.LogStreamOptions") - proto.RegisterType((*LogStreamMessage)(nil), "pb.LogStreamMessage") - proto.RegisterType((*ExecuteContainerOptions)(nil), "pb.ExecuteContainerOptions") -} - -func init() { proto.RegisterFile("core.proto", fileDescriptor_f7e43720d1edc0fe) } - -var fileDescriptor_f7e43720d1edc0fe = []byte{ - // 4994 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3c, 0x4d, 0x6f, 0x1c, 0xc7, - 0x72, 0xdc, 0x5d, 0xee, 0x57, 0xed, 0x07, 0x97, 0x2d, 0x8a, 0x5e, 0x8f, 0x64, 0x8b, 0x1c, 0xe5, - 0x59, 0xb2, 0x9f, 0x4d, 0xcb, 0xb2, 0x2d, 0x7f, 0xcb, 0x8f, 0x22, 0x29, 0x89, 0x78, 0x94, 0xc5, - 0x37, 0x2b, 0xdb, 0x09, 0x72, 0xd8, 0x0c, 0x67, 0x9a, 0xcb, 0x81, 0x77, 0x67, 0xc6, 0x33, 0xb3, - 0x94, 0x78, 0xcc, 0xe9, 0xdd, 0x92, 0x9c, 0x12, 0x24, 0x97, 0x5c, 0x02, 0xe4, 0x96, 0x1c, 0x02, - 0x04, 0xc9, 0x21, 0x40, 0x80, 0xfc, 0x85, 0x1c, 0x93, 0x3f, 0xf0, 0x8e, 0x01, 0x02, 0xe4, 0x12, - 0x20, 0xe8, 0xea, 0x8f, 0xe9, 0x99, 0x9d, 0x25, 0xb9, 0xd2, 0xfb, 0x00, 0x72, 0x62, 0x77, 0x75, - 0x75, 0x4d, 0x77, 0x75, 0x75, 0x7d, 0x75, 0x2d, 0x01, 0x9c, 0x20, 0xa2, 0x5b, 0x61, 0x14, 0x24, - 0x01, 0x29, 0x87, 0x47, 0x66, 0x1d, 0xaa, 0x7b, 0x93, 0x30, 0x39, 0x33, 0xff, 0xb2, 0x04, 0x8d, - 0x9d, 0x20, 0xa2, 0xfb, 0xfe, 0x71, 0x40, 0xfa, 0x50, 0x3f, 0xa5, 0x51, 0xec, 0x05, 0x7e, 0xbf, - 0xb4, 0x51, 0xba, 0xdd, 0xb4, 0x64, 0x97, 0x8d, 0x44, 0xf4, 0xd4, 0x8b, 0x03, 0xbf, 0x5f, 0xe6, - 0x23, 0xa2, 0x4b, 0x5e, 0x87, 0xc6, 0xd1, 0xd4, 0x1b, 0xbb, 0x43, 0x3b, 0xe9, 0x57, 0xf8, 0x10, - 0xf6, 0xb7, 0x13, 0xf2, 0x13, 0xe8, 0x8e, 0x82, 0xb1, 0xed, 0x8f, 0x86, 0x92, 0xea, 0x32, 0x22, - 0x74, 0x38, 0xf4, 0x3b, 0x41, 0xfb, 0x35, 0xa8, 0x07, 0xf1, 0xd0, 0x8e, 0x9c, 0x93, 0x7e, 0x15, - 0xc7, 0x6b, 0x41, 0xbc, 0x1d, 0x39, 0x27, 0xe6, 0x1f, 0x42, 0x67, 0x40, 0xa3, 0x53, 0xcf, 0xa1, - 0x83, 0xc4, 0x4e, 0xa6, 0x31, 0xb9, 0x0e, 0x4d, 0xdb, 0x75, 0x23, 0x1a, 0xc7, 0x34, 0xee, 0x97, - 0x36, 0x2a, 0xb7, 0x9b, 0x56, 0x0a, 0x20, 0xef, 0x02, 0xf1, 0xfc, 0x84, 0x46, 0xa7, 0xf6, 0x78, - 0xe8, 0xf9, 0xc3, 0x98, 0x3a, 0x81, 0xef, 0xe2, 0x72, 0x2b, 0x56, 0x4f, 0x8e, 0xec, 0xfb, 0x03, - 0x84, 0x9b, 0xff, 0x5b, 0x82, 0xab, 0x07, 0x5e, 0x9c, 0xec, 0x04, 0x7e, 0x62, 0x7b, 0x3e, 0x8d, - 0xe2, 0xa7, 0x61, 0xe2, 0x05, 0x7e, 0xcc, 0xf6, 0x6a, 0x87, 0xa1, 0x6f, 0x4f, 0xa8, 0xe4, 0x82, - 0xe8, 0x92, 0x37, 0x01, 0xa8, 0x9f, 0x44, 0x67, 0x61, 0xe0, 0xf9, 0x89, 0x60, 0x84, 0x06, 0x21, - 0x06, 0x34, 0xfc, 0xc0, 0xa5, 0x38, 0x95, 0xf3, 0x42, 0xf5, 0xc9, 0x57, 0x50, 0x1b, 0xdb, 0x47, - 0x74, 0x1c, 0xf7, 0x97, 0x37, 0x2a, 0xb7, 0x5b, 0x77, 0x7f, 0xb2, 0x15, 0x1e, 0x6d, 0x15, 0x2e, - 0x60, 0xeb, 0x00, 0xf1, 0xf6, 0x18, 0x5d, 0x4b, 0x4c, 0x22, 0x6b, 0x50, 0x1d, 0x7b, 0x13, 0x2f, - 0x41, 0x16, 0x55, 0x2c, 0xde, 0x31, 0x3e, 0x83, 0x96, 0x86, 0x4c, 0x7a, 0x50, 0xf9, 0x81, 0x9e, - 0x89, 0x55, 0xb3, 0x26, 0x9b, 0x76, 0x6a, 0x8f, 0xa7, 0x54, 0x2c, 0x96, 0x77, 0x3e, 0x2f, 0x7f, - 0x5a, 0x32, 0xdf, 0x83, 0xca, 0x61, 0xe0, 0x12, 0x02, 0xcb, 0xda, 0x4e, 0xb1, 0xcd, 0x60, 0x2e, - 0x8d, 0x1d, 0x31, 0x07, 0xdb, 0xe6, 0x4d, 0x58, 0x3e, 0x0c, 0xdc, 0x98, 0x5c, 0x83, 0xe5, 0x30, - 0x70, 0x39, 0xf7, 0x5b, 0x77, 0xeb, 0x6c, 0x13, 0x87, 0x81, 0x6b, 0x21, 0xd0, 0xfc, 0xcf, 0x1a, - 0xb4, 0x58, 0x8f, 0xc6, 0xc1, 0x34, 0x72, 0x68, 0x21, 0xf1, 0x1d, 0x68, 0x3b, 0xe1, 0x74, 0x18, - 0xd2, 0xc8, 0xa1, 0x7e, 0x12, 0xf7, 0xcb, 0x48, 0x68, 0x43, 0x12, 0x12, 0x53, 0xb7, 0x76, 0xc2, - 0xe9, 0xa1, 0x40, 0xe1, 0x8c, 0x68, 0x39, 0x29, 0x84, 0x1c, 0xc0, 0xca, 0x84, 0x4e, 0x82, 0xe8, - 0x2c, 0xa5, 0x53, 0x41, 0x3a, 0x37, 0xf3, 0x74, 0x9e, 0x20, 0x5a, 0x96, 0x54, 0x77, 0x92, 0x01, - 0x92, 0xc7, 0xd0, 0x39, 0xa5, 0x91, 0x77, 0xec, 0x39, 0x36, 0x1e, 0x80, 0x38, 0x21, 0x33, 0x4f, - 0xeb, 0x3b, 0x1d, 0x89, 0x93, 0xca, 0x4e, 0x24, 0xf7, 0xa0, 0xee, 0xd2, 0xc4, 0xf6, 0xc6, 0x71, - 0xbf, 0x8a, 0x34, 0xae, 0xe7, 0x69, 0xec, 0xf2, 0x61, 0x3e, 0x5b, 0x22, 0x93, 0xa7, 0xd0, 0x8b, - 0x93, 0x20, 0xb2, 0x47, 0x34, 0xdd, 0x50, 0x0d, 0x09, 0xfc, 0x5e, 0x9e, 0xc0, 0x80, 0xe3, 0x65, - 0x77, 0xb4, 0x12, 0x67, 0xa1, 0x8c, 0x41, 0xa7, 0xc1, 0x78, 0x3a, 0xd1, 0xe8, 0xd5, 0x8b, 0x19, - 0xf4, 0x1d, 0xa2, 0xe5, 0x18, 0x74, 0x9a, 0x01, 0x1a, 0xf7, 0xa1, 0x97, 0x3f, 0x8f, 0x8b, 0x64, - 0xad, 0xa4, 0xc9, 0x9a, 0xb1, 0x0d, 0x57, 0x0a, 0xce, 0x61, 0x21, 0x12, 0x3f, 0x03, 0x32, 0xcb, - 0xfe, 0x8b, 0x28, 0x34, 0x74, 0x0a, 0x9f, 0x43, 0x5b, 0x67, 0xfe, 0x22, 0x97, 0xc5, 0x78, 0x00, - 0x6b, 0x45, 0x7c, 0x5f, 0x94, 0x09, 0x05, 0xbc, 0x5e, 0x84, 0x84, 0xf9, 0x3f, 0x25, 0x68, 0x7f, - 0x13, 0xb8, 0xf4, 0xdc, 0x0b, 0x76, 0x03, 0x5a, 0xda, 0x05, 0x13, 0x44, 0x20, 0xbd, 0x3d, 0x4c, - 0x2d, 0x67, 0x2f, 0x0f, 0xea, 0xaa, 0x92, 0xd5, 0xc9, 0x5c, 0x0b, 0x62, 0x42, 0x5b, 0x17, 0x6e, - 0xd4, 0xdd, 0x0d, 0x2b, 0x03, 0x63, 0xaa, 0x52, 0x97, 0xf7, 0x66, 0x2a, 0xd1, 0xb7, 0x60, 0x25, - 0x27, 0xd1, 0xfd, 0x1a, 0x7e, 0xa5, 0x9b, 0x15, 0x55, 0xb6, 0x9a, 0xac, 0xa4, 0xf6, 0xeb, 0x7c, - 0x35, 0x19, 0x19, 0x34, 0x1f, 0x02, 0x61, 0xca, 0xf2, 0x1b, 0x9a, 0x3c, 0x0f, 0xa2, 0x1f, 0x34, - 0x55, 0x1d, 0x06, 0xae, 0xae, 0xaa, 0x45, 0x97, 0xac, 0x43, 0xcd, 0x8d, 0xbc, 0x53, 0x1a, 0x89, - 0xc3, 0x14, 0x3d, 0xf3, 0x47, 0xb8, 0xba, 0x13, 0xf8, 0x3e, 0x75, 0x0a, 0x48, 0xf9, 0x1c, 0x22, - 0x49, 0x89, 0x2e, 0x23, 0x95, 0xd8, 0xd1, 0x88, 0x4a, 0x8d, 0x2f, 0x7a, 0x8c, 0xf9, 0x5e, 0x78, - 0xfa, 0x91, 0xd0, 0xf4, 0xd8, 0x16, 0xb0, 0x7b, 0xc2, 0xd0, 0x61, 0xdb, 0x3c, 0x82, 0xfe, 0xae, - 0x17, 0x3b, 0xbf, 0xa6, 0xaf, 0xae, 0x41, 0xf5, 0x38, 0x88, 0x1c, 0x6e, 0x60, 0x1a, 0x16, 0xef, - 0x98, 0x9f, 0x40, 0x5d, 0x50, 0x2e, 0x94, 0x89, 0x3e, 0xd4, 0xe3, 0xe9, 0x91, 0x4f, 0x85, 0xbe, - 0x6d, 0x5a, 0xb2, 0x6b, 0x7e, 0x08, 0x0d, 0x31, 0x91, 0x9d, 0x59, 0x43, 0x7c, 0x5d, 0xea, 0xf7, - 0x16, 0xd3, 0x16, 0x62, 0xdc, 0x52, 0x83, 0xe6, 0xaf, 0x1a, 0xb0, 0xcc, 0xe4, 0xb0, 0xf0, 0x5b, - 0x06, 0x34, 0xa8, 0xef, 0xea, 0x26, 0x52, 0xf5, 0xf5, 0xf3, 0xaa, 0x64, 0xcf, 0xeb, 0x26, 0x54, - 0x9c, 0x70, 0x2a, 0x34, 0xef, 0x2a, 0x7e, 0x36, 0x70, 0xd1, 0x0c, 0x70, 0x95, 0xc4, 0x46, 0x99, - 0xaf, 0xc1, 0x44, 0x7b, 0x1a, 0x53, 0x17, 0xed, 0x60, 0xc9, 0xaa, 0x3b, 0xe1, 0xf4, 0xdb, 0x98, - 0xba, 0x8c, 0x5d, 0x5c, 0x7c, 0x51, 0xcc, 0x2a, 0x96, 0xe8, 0xb1, 0xdb, 0x20, 0x84, 0x1d, 0x67, - 0xd5, 0x71, 0x10, 0x38, 0x08, 0x27, 0x32, 0x9f, 0xe2, 0xd4, 0xf6, 0xc6, 0xf6, 0xd1, 0x98, 0xf6, - 0x1b, 0xc8, 0xd3, 0x14, 0x40, 0xde, 0x55, 0x56, 0xbb, 0x89, 0x2b, 0x5b, 0x53, 0x2b, 0x2b, 0x32, - 0xd2, 0x37, 0xa0, 0xe5, 0xf9, 0x5e, 0x32, 0x14, 0x2b, 0x01, 0xfe, 0x31, 0x06, 0xe2, 0xea, 0x8f, - 0xdc, 0x81, 0x06, 0x22, 0xb0, 0xad, 0xb6, 0x90, 0xe0, 0x55, 0x45, 0x70, 0xdf, 0xf7, 0x12, 0xb5, - 0xdd, 0xba, 0xc7, 0x7b, 0x28, 0x50, 0xfe, 0x71, 0xd0, 0x6f, 0x0b, 0x81, 0x62, 0x6e, 0xda, 0x5b, - 0xb0, 0xec, 0x4f, 0x27, 0x76, 0xbf, 0x83, 0x14, 0x88, 0xa2, 0xf0, 0xcd, 0x74, 0x62, 0xf3, 0xe9, - 0x38, 0x4e, 0x3e, 0x83, 0x16, 0xfb, 0x2b, 0x97, 0xd3, 0x45, 0xf4, 0x7e, 0x06, 0x9d, 0xaf, 0x8b, - 0x4f, 0x02, 0x5f, 0x01, 0x50, 0x60, 0xf8, 0x3d, 0xed, 0xaf, 0xe0, 0x2e, 0x64, 0x97, 0x6c, 0x42, - 0x5b, 0x5e, 0x6c, 0xe4, 0x68, 0x0f, 0x87, 0x5b, 0x02, 0x86, 0x2c, 0xdd, 0x84, 0x36, 0xee, 0x52, - 0x52, 0x58, 0xe5, 0x28, 0x0c, 0x26, 0xb4, 0x28, 0x5b, 0x1a, 0xa2, 0xf0, 0x4b, 0xde, 0x27, 0xb9, - 0xa5, 0x31, 0x5e, 0x70, 0x65, 0x29, 0x96, 0xe6, 0x29, 0x00, 0x3b, 0x12, 0x31, 0xeb, 0x4a, 0xee, - 0x48, 0xf4, 0x19, 0x02, 0x87, 0x1d, 0x89, 0x50, 0x2f, 0xb8, 0xda, 0x35, 0x7e, 0x24, 0x1c, 0xc4, - 0x16, 0x6b, 0xdc, 0x83, 0x86, 0xe4, 0xfa, 0x45, 0xba, 0xb8, 0xaa, 0xab, 0xf3, 0x97, 0x77, 0xbd, - 0x98, 0x25, 0xd2, 0x0f, 0x7b, 0xa1, 0xcf, 0x7e, 0x02, 0x4d, 0x75, 0xcc, 0x0b, 0x7d, 0xf4, 0x2b, - 0x58, 0xc9, 0x1d, 0xf8, 0x45, 0xd3, 0x2b, 0xb9, 0xe9, 0xb9, 0x43, 0x59, 0x68, 0xfa, 0x67, 0xd0, - 0x7a, 0xc9, 0xa9, 0xe6, 0x2d, 0xa8, 0xb2, 0xd3, 0x8d, 0xc9, 0x9b, 0x50, 0x65, 0xde, 0xb4, 0xd4, - 0x4d, 0x0d, 0x79, 0xee, 0x16, 0x07, 0x9b, 0x7b, 0xd0, 0x61, 0xdd, 0x6d, 0x75, 0x79, 0x75, 0x77, - 0xbc, 0x94, 0x73, 0xc7, 0x35, 0x4d, 0x54, 0xce, 0x68, 0x22, 0xf3, 0xdf, 0x6a, 0xd0, 0x1d, 0xd0, - 0x84, 0x91, 0x92, 0x5a, 0xfa, 0x3c, 0x42, 0x6f, 0x03, 0xc4, 0x18, 0x9d, 0x0c, 0x83, 0x90, 0x2b, - 0xbc, 0xee, 0x5d, 0x60, 0x4b, 0x7b, 0x16, 0x79, 0x4f, 0xc3, 0xc4, 0x6a, 0xf2, 0xd1, 0xa7, 0x61, - 0x42, 0xbe, 0x82, 0xa6, 0x4b, 0xc7, 0x89, 0x8d, 0xd7, 0xbf, 0x92, 0xfa, 0xbd, 0xd9, 0xaf, 0x6d, - 0xed, 0x32, 0x1c, 0xa5, 0x09, 0x1a, 0xae, 0xe8, 0xb2, 0x6b, 0xc5, 0xa7, 0x8b, 0xfb, 0xbc, 0xcc, - 0xaf, 0x15, 0xc2, 0xc4, 0xb5, 0xbd, 0x09, 0x1d, 0x8e, 0x22, 0xaf, 0x1e, 0x8f, 0x16, 0xf8, 0x3c, - 0x79, 0xf7, 0x06, 0xb0, 0xca, 0x91, 0x74, 0xe5, 0xc0, 0xbd, 0xcd, 0x5b, 0xf3, 0x96, 0x93, 0xd7, - 0x15, 0x2b, 0x6e, 0x16, 0x4a, 0xee, 0x08, 0x9d, 0x54, 0x4f, 0xdd, 0xde, 0x1c, 0x9d, 0xbc, 0x76, - 0xba, 0xa7, 0x54, 0x6b, 0x03, 0xe7, 0xbc, 0x59, 0x30, 0xa7, 0x48, 0xc9, 0x3e, 0x94, 0x6c, 0x10, - 0x5a, 0xa0, 0x99, 0xfa, 0xb5, 0x45, 0x2b, 0xd7, 0x95, 0x02, 0xe7, 0x95, 0xd0, 0x23, 0xb7, 0x60, - 0xc5, 0x51, 0xa1, 0xd7, 0xd0, 0x0d, 0x9e, 0xfb, 0xa8, 0xb0, 0x1b, 0x56, 0x37, 0x05, 0xef, 0x06, - 0xcf, 0x7d, 0xe3, 0x0b, 0xe8, 0x64, 0x8e, 0x64, 0xa1, 0xfb, 0xfa, 0x00, 0xd6, 0x8a, 0x18, 0xb8, - 0xd0, 0xe5, 0x79, 0xe9, 0x3b, 0xff, 0x0a, 0x3a, 0xea, 0x3e, 0xf4, 0xf2, 0xec, 0x5b, 0xe8, 0xd6, - 0xfe, 0x4b, 0x03, 0x9a, 0x2a, 0xb2, 0x25, 0x5d, 0x28, 0x7b, 0xae, 0x98, 0x58, 0xf6, 0xdc, 0xf9, - 0xb7, 0xef, 0xdc, 0x10, 0x5a, 0x7a, 0x1b, 0xcb, 0x9a, 0xb7, 0x71, 0x9b, 0xfb, 0x0d, 0x3c, 0xda, - 0x5a, 0x67, 0x42, 0xa0, 0xbe, 0x9a, 0x73, 0x1e, 0xd6, 0xa0, 0xfa, 0xe3, 0x34, 0x48, 0x6c, 0xe1, - 0x87, 0xf2, 0x8e, 0xe6, 0x37, 0xd4, 0x33, 0x7e, 0xc3, 0x9b, 0x00, 0x61, 0xe4, 0x9d, 0x7a, 0x63, - 0x3a, 0xa2, 0xae, 0xf0, 0x0b, 0x34, 0x08, 0xf9, 0x20, 0xe7, 0x18, 0xbc, 0x9e, 0xfd, 0x74, 0x91, - 0xe0, 0x7e, 0x04, 0xf5, 0x70, 0x7a, 0x34, 0xf6, 0xe2, 0x93, 0x3e, 0xe0, 0x1c, 0x23, 0x3b, 0xe7, - 0x90, 0x0f, 0x0a, 0x07, 0x40, 0xa0, 0xb2, 0x65, 0x7b, 0x13, 0x76, 0x95, 0x5b, 0xfc, 0x88, 0xb0, - 0xa3, 0xdb, 0xe7, 0x76, 0xd6, 0x3e, 0xff, 0x14, 0x6a, 0x5c, 0xe3, 0xf4, 0x3b, 0x1b, 0xa5, 0xdb, - 0xad, 0xbb, 0x57, 0x32, 0x1f, 0xe1, 0x89, 0x14, 0x4b, 0xa0, 0x60, 0xc2, 0x07, 0x0f, 0x38, 0x46, - 0xef, 0xa0, 0x69, 0xc9, 0x2e, 0xb9, 0xaf, 0xec, 0x66, 0x38, 0xb6, 0xfd, 0xfe, 0x0a, 0x2e, 0xf8, - 0x8d, 0xec, 0x82, 0x45, 0x38, 0x33, 0xb6, 0x7d, 0x61, 0xa5, 0x4f, 0x15, 0x80, 0x69, 0x22, 0x64, - 0xf0, 0x30, 0xa2, 0x3f, 0x4e, 0x69, 0x9c, 0xa0, 0x9f, 0x50, 0xb2, 0xda, 0x08, 0xb4, 0x38, 0x4c, - 0x8b, 0x44, 0x24, 0x16, 0x77, 0x15, 0x44, 0x24, 0x22, 0xd1, 0xb4, 0x58, 0x42, 0xe2, 0x11, 0xc4, - 0x93, 0xb1, 0x84, 0x86, 0x28, 0xd6, 0xaf, 0x10, 0xaf, 0xe0, 0xb6, 0x44, 0x88, 0x11, 0x4b, 0xc4, - 0x67, 0x70, 0x45, 0xdb, 0x9d, 0x42, 0x5e, 0x4b, 0x43, 0xee, 0xa2, 0x5d, 0x8a, 0xd9, 0x7c, 0xb3, - 0xab, 0xa7, 0x79, 0xf8, 0xef, 0xc8, 0x95, 0xd0, 0xc5, 0x66, 0xa1, 0xb9, 0xfb, 0xb0, 0x92, 0x3b, - 0xc1, 0x82, 0xe9, 0x1b, 0xfa, 0xf4, 0x16, 0xb7, 0x6c, 0x7c, 0x96, 0x4e, 0xea, 0x10, 0xd6, 0x8b, - 0xd9, 0xf4, 0xb2, 0x14, 0xcd, 0x3f, 0x2e, 0xc3, 0x4a, 0x4e, 0x6a, 0x8b, 0xb4, 0x48, 0x34, 0xf5, - 0x7d, 0xcf, 0x1f, 0x89, 0x68, 0x5f, 0x76, 0xd9, 0xc8, 0x09, 0xb5, 0xc7, 0xc9, 0xc9, 0x99, 0x08, - 0x93, 0x64, 0x97, 0x7c, 0xa5, 0xc5, 0x38, 0x3c, 0xd8, 0xd8, 0x2c, 0xb8, 0x20, 0x32, 0xe6, 0x11, - 0x37, 0x58, 0x4d, 0x61, 0xd1, 0x02, 0x7d, 0x91, 0x50, 0x1f, 0xb3, 0x99, 0xcc, 0xb8, 0xb6, 0xad, - 0x14, 0xc0, 0x36, 0x9b, 0x24, 0x63, 0x11, 0x81, 0xb0, 0x26, 0xb3, 0x1d, 0x19, 0x52, 0x0b, 0xa5, - 0xe8, 0xbe, 0x86, 0x5e, 0x9a, 0x1c, 0x14, 0x3c, 0x48, 0xaf, 0x37, 0xf7, 0x82, 0xce, 0xbb, 0xde, - 0xe6, 0x3f, 0x96, 0xe0, 0x7a, 0x6e, 0x6c, 0x90, 0x44, 0xd4, 0x9e, 0x3c, 0xa1, 0x71, 0xcc, 0x94, - 0x45, 0x9e, 0xa3, 0x3f, 0x85, 0xa6, 0x32, 0x7e, 0xe2, 0x7c, 0x3a, 0x99, 0x0f, 0x58, 0xe9, 0xb8, - 0xb6, 0x94, 0xca, 0xc5, 0x9a, 0x66, 0x0d, 0xaa, 0x34, 0x8a, 0x82, 0x48, 0x28, 0x6f, 0xde, 0xc1, - 0x28, 0x9d, 0x8e, 0x69, 0xc2, 0x1d, 0x95, 0x86, 0x25, 0x7a, 0xe6, 0x3e, 0x18, 0x03, 0x9a, 0xe4, - 0x37, 0x2f, 0xdd, 0xb1, 0x85, 0x78, 0xf0, 0xdf, 0xf3, 0x78, 0xf0, 0x9b, 0x4d, 0xf7, 0xee, 0xe6, - 0xd2, 0xbd, 0xef, 0x16, 0xac, 0x31, 0xb3, 0x8e, 0x22, 0x93, 0xf1, 0x2a, 0xf9, 0xdd, 0x2f, 0x00, - 0x52, 0xfe, 0x91, 0xf7, 0x00, 0x52, 0xaf, 0x46, 0xb0, 0x2d, 0x77, 0xb2, 0x1a, 0x82, 0xf9, 0x06, - 0xb4, 0xd4, 0xc0, 0xfe, 0x6e, 0x5e, 0x4c, 0xcc, 0x0d, 0x68, 0x6b, 0xc3, 0x31, 0x5b, 0x97, 0xe7, - 0xca, 0x8c, 0x3c, 0x6b, 0x9a, 0xcf, 0x60, 0xdd, 0xa2, 0x93, 0xe0, 0x94, 0x2a, 0x3c, 0xc9, 0xee, - 0x19, 0xdc, 0x34, 0xa3, 0x51, 0xd6, 0x32, 0x1a, 0xcc, 0xd8, 0xc7, 0x09, 0x0d, 0x91, 0xb1, 0x55, - 0x0b, 0xdb, 0xe6, 0x16, 0x18, 0xbb, 0x5e, 0x1c, 0x07, 0x8e, 0x67, 0x27, 0x97, 0xa0, 0x6c, 0xfe, - 0xaa, 0x02, 0x5d, 0x8b, 0xda, 0xe3, 0x71, 0xe0, 0xcc, 0xff, 0x7c, 0x8f, 0x7b, 0x10, 0x3c, 0x4f, - 0x86, 0x9e, 0x42, 0xea, 0x13, 0x54, 0x32, 0x3e, 0x81, 0x66, 0x2d, 0x97, 0xb3, 0xd6, 0xf2, 0x5d, - 0x68, 0x1f, 0x79, 0xbe, 0xcb, 0x1c, 0x7b, 0x0c, 0x03, 0xaa, 0x33, 0x61, 0x00, 0xb0, 0xf1, 0x9d, - 0x70, 0xca, 0xe2, 0x80, 0x8f, 0xa0, 0x27, 0xcc, 0x1e, 0x66, 0xf1, 0x71, 0x46, 0x6d, 0x66, 0x86, - 0x30, 0x8d, 0x07, 0x0c, 0x85, 0xcd, 0x12, 0x79, 0x3d, 0x69, 0xab, 0xea, 0x2a, 0xaf, 0x27, 0x8d, - 0xda, 0x35, 0x68, 0x32, 0x04, 0xfe, 0x4c, 0xd0, 0xc0, 0xe1, 0x86, 0x13, 0x4e, 0x91, 0x40, 0x81, - 0xa9, 0x6d, 0x16, 0x99, 0xda, 0x4d, 0x68, 0xeb, 0x4b, 0x13, 0x29, 0x8c, 0x96, 0xb6, 0x94, 0x22, - 0x6b, 0xdc, 0x2a, 0xb4, 0xc6, 0x37, 0xa1, 0x23, 0x11, 0x39, 0x31, 0xee, 0xa9, 0xc8, 0xf4, 0x81, - 0x5a, 0x97, 0xb0, 0xc4, 0x92, 0x58, 0x07, 0x59, 0x2b, 0xd2, 0x7f, 0xda, 0xba, 0x04, 0x1a, 0x27, - 0xc5, 0xbd, 0x15, 0xe1, 0xa2, 0x20, 0x25, 0xf3, 0x13, 0xe8, 0x6c, 0xbb, 0xee, 0x61, 0xe0, 0xca, - 0xa3, 0xbe, 0xec, 0xd3, 0xc6, 0x5b, 0xd0, 0xe3, 0xb2, 0x7a, 0xfe, 0x5c, 0xf3, 0x26, 0x74, 0x1e, - 0xd1, 0xe4, 0x02, 0xa4, 0x7f, 0xaf, 0x42, 0x77, 0xdb, 0x75, 0x2f, 0x1b, 0x3d, 0xbe, 0x5c, 0xb2, - 0xac, 0x0b, 0x65, 0xc7, 0x16, 0x9a, 0xb4, 0xec, 0xd8, 0x6c, 0x21, 0x0e, 0x8d, 0x12, 0xf1, 0x7c, - 0x86, 0x6d, 0xa9, 0x2b, 0x6a, 0xa9, 0xae, 0x10, 0x82, 0x5e, 0xc7, 0x0b, 0x25, 0x5d, 0xe2, 0xf8, - 0xc4, 0x8e, 0x78, 0xde, 0xab, 0x6a, 0xf1, 0x8e, 0x26, 0xfe, 0xcd, 0x8c, 0xf8, 0xa7, 0x01, 0x1b, - 0xa4, 0x01, 0x5b, 0x76, 0xaf, 0x85, 0x7e, 0xaf, 0x0c, 0x0d, 0x5b, 0x69, 0x68, 0x98, 0x9b, 0x95, - 0x0f, 0x0d, 0x77, 0xb2, 0x89, 0xab, 0x76, 0xfa, 0x1c, 0x53, 0x30, 0xf1, 0x12, 0x29, 0xac, 0x4e, - 0xd6, 0x45, 0xfe, 0x19, 0x08, 0x4f, 0x75, 0x38, 0xb1, 0x43, 0x91, 0x16, 0xdb, 0x2c, 0xa0, 0xce, - 0xbd, 0x92, 0x27, 0x76, 0xc8, 0x89, 0x37, 0x4f, 0x65, 0xff, 0x55, 0x3c, 0xb6, 0xdf, 0x55, 0x02, - 0xe7, 0x4b, 0xe8, 0x66, 0xf7, 0xb3, 0x50, 0x38, 0xf7, 0x3e, 0xac, 0xf2, 0x3b, 0x72, 0x49, 0xc1, - 0x36, 0xff, 0xba, 0x04, 0xdd, 0x47, 0x97, 0xcf, 0xa2, 0xa4, 0xb2, 0x55, 0x4e, 0x65, 0xeb, 0xd1, - 0x85, 0xc9, 0x80, 0x57, 0x31, 0x90, 0x16, 0xac, 0x8b, 0x0f, 0xc8, 0xe7, 0x14, 0xb9, 0xd0, 0xb7, - 0x60, 0x39, 0x08, 0x93, 0x18, 0xc9, 0x88, 0xfc, 0x6a, 0x76, 0x29, 0x16, 0x8e, 0xb3, 0xaf, 0x1d, - 0x7b, 0x2f, 0x84, 0xd9, 0x62, 0x4d, 0xf3, 0x1f, 0x4a, 0xd0, 0xc3, 0x67, 0x8a, 0xc0, 0xa5, 0xf1, - 0xc5, 0x8f, 0x14, 0x3d, 0xa8, 0xd8, 0xe3, 0xb1, 0x24, 0x60, 0x8f, 0xc7, 0xe4, 0x53, 0xc5, 0x07, - 0x2d, 0x3f, 0x94, 0xa7, 0xf8, 0xeb, 0xe6, 0xc4, 0x7f, 0x54, 0xa1, 0xfa, 0x60, 0xea, 0x8d, 0xf1, - 0x35, 0xf8, 0xc8, 0x8e, 0x95, 0x46, 0x63, 0x6d, 0x06, 0x8b, 0x68, 0x18, 0x48, 0x95, 0xc9, 0xda, - 0x7a, 0xa1, 0x40, 0x25, 0x5b, 0x28, 0xd0, 0x83, 0x8a, 0xeb, 0x49, 0x2f, 0x8f, 0x35, 0x99, 0xcb, - 0x1c, 0x4f, 0x8f, 0x26, 0x81, 0x3b, 0x1d, 0x4b, 0x37, 0x2f, 0x05, 0x30, 0xa1, 0x70, 0x82, 0xc9, - 0xc4, 0xf6, 0x5d, 0xfe, 0xe2, 0xd9, 0xb4, 0x54, 0x9f, 0xdc, 0x82, 0x65, 0xea, 0x9f, 0xca, 0x97, - 0x4b, 0xf4, 0xf2, 0x70, 0x99, 0x5b, 0x7b, 0xfe, 0xa9, 0xd8, 0x3d, 0x22, 0x30, 0x44, 0x3b, 0x1a, - 0xc9, 0x44, 0x92, 0x86, 0xb8, 0x1d, 0x8d, 0x24, 0x22, 0x43, 0x20, 0xef, 0xe5, 0xa2, 0xf6, 0xab, - 0x29, 0x6a, 0x91, 0xe6, 0xba, 0x07, 0x4d, 0x3b, 0x4a, 0xbc, 0x63, 0xdb, 0x49, 0xa4, 0xd2, 0xeb, - 0xeb, 0xc4, 0xc5, 0x90, 0x50, 0x0f, 0x0a, 0x95, 0xbc, 0x03, 0x55, 0xc7, 0x76, 0x4e, 0xa8, 0x50, - 0x79, 0x6b, 0xe9, 0x9c, 0x1d, 0x06, 0xe6, 0xf8, 0x1c, 0x85, 0x99, 0xf5, 0x38, 0x09, 0xc2, 0x61, - 0xec, 0x8d, 0x7c, 0x7b, 0x2c, 0xf2, 0xfc, 0xc0, 0x40, 0x03, 0x84, 0x30, 0x0e, 0xc5, 0xd4, 0x99, - 0x46, 0x5e, 0x72, 0x86, 0x8a, 0xac, 0x61, 0xa9, 0x3e, 0x53, 0x26, 0x8a, 0x17, 0x8b, 0x6a, 0x21, - 0xc5, 0x9b, 0xdf, 0x56, 0x4a, 0xe9, 0x4b, 0xe8, 0x66, 0x59, 0xb6, 0xd0, 0xec, 0x4f, 0x01, 0x52, - 0xe6, 0x2d, 0x24, 0xde, 0x7f, 0x5e, 0x82, 0x1a, 0x72, 0x3f, 0x66, 0xa6, 0x2d, 0x4e, 0xec, 0x91, - 0xaa, 0x1e, 0x11, 0x3d, 0xb2, 0x05, 0x35, 0x2c, 0x5a, 0x91, 0xea, 0x67, 0x5d, 0x9d, 0x58, 0x2c, - 0xfe, 0x08, 0xc1, 0xe0, 0x58, 0xc6, 0x2e, 0xb4, 0x34, 0x70, 0xc1, 0x6a, 0x6e, 0x64, 0x83, 0xdc, - 0xa6, 0xa2, 0xa7, 0x2f, 0xec, 0x6f, 0xca, 0xb0, 0x8a, 0xc0, 0xfd, 0x89, 0x3d, 0xa2, 0x17, 0xb8, - 0x2d, 0xd3, 0x58, 0xbd, 0x65, 0x62, 0x9b, 0x7d, 0x74, 0xea, 0xb9, 0xc2, 0x3f, 0x66, 0x4d, 0x86, - 0x95, 0xd8, 0x23, 0xe9, 0x9c, 0x62, 0x9b, 0x98, 0x6a, 0x67, 0xd5, 0x34, 0xdc, 0xe6, 0x6b, 0x97, - 0xbb, 0xc1, 0xb0, 0xd5, 0x8e, 0xd0, 0x55, 0x68, 0x5b, 0xac, 0x49, 0x76, 0xa1, 0xcd, 0x8b, 0x7a, - 0x26, 0x34, 0x39, 0x09, 0xf8, 0xb3, 0x59, 0x97, 0xdb, 0xc8, 0x99, 0x05, 0x73, 0xc8, 0x13, 0x44, - 0xb4, 0x5a, 0x47, 0x69, 0x87, 0xbc, 0x0e, 0x0d, 0xfa, 0xc2, 0x8b, 0x93, 0xa1, 0xc7, 0x33, 0x68, - 0x4d, 0xab, 0x8e, 0xfd, 0x7d, 0xd7, 0x7c, 0x47, 0x30, 0x50, 0x60, 0xd6, 0xa1, 0x32, 0xd8, 0x79, - 0xd2, 0x5b, 0x62, 0x0d, 0x6b, 0xfb, 0xfb, 0x5e, 0x89, 0x34, 0xa1, 0xba, 0xf7, 0xfb, 0xfb, 0x83, - 0x67, 0xbd, 0xb2, 0x49, 0xa1, 0xf5, 0x38, 0x08, 0xd4, 0x93, 0xe9, 0x0d, 0x68, 0xd9, 0xc7, 0x09, - 0x8d, 0x86, 0x71, 0x62, 0x47, 0x89, 0x38, 0x48, 0x40, 0xd0, 0x80, 0x41, 0x18, 0xc2, 0x11, 0x3d, - 0x0e, 0x22, 0x3a, 0x64, 0xb7, 0x48, 0x3c, 0x78, 0x02, 0x07, 0x0d, 0x92, 0x20, 0x9c, 0xf3, 0x84, - 0x9a, 0x00, 0x79, 0x8c, 0x49, 0x82, 0x9d, 0x13, 0xea, 0xa8, 0xaf, 0x5d, 0x83, 0x66, 0xe2, 0x84, - 0xc3, 0x30, 0x88, 0x12, 0x29, 0x34, 0x8d, 0xc4, 0x09, 0x0f, 0x59, 0x9f, 0x0d, 0x9e, 0x24, 0x09, - 0x1f, 0x95, 0xee, 0x1b, 0x03, 0xb0, 0x51, 0x3c, 0x9f, 0x68, 0x2c, 0xf4, 0x23, 0x6b, 0xa2, 0x9b, - 0x16, 0xb8, 0x3c, 0x7f, 0x59, 0xb5, 0xb0, 0x6d, 0xfe, 0x69, 0x09, 0xe0, 0x20, 0x18, 0x69, 0x87, - 0x9f, 0x9c, 0x85, 0xea, 0xf0, 0x59, 0x9b, 0xdc, 0x85, 0x9a, 0x13, 0xf8, 0xc7, 0xde, 0x48, 0x08, - 0x27, 0xa6, 0x0d, 0xd3, 0x39, 0x2c, 0x84, 0x3b, 0xf6, 0x46, 0x42, 0x40, 0x39, 0x26, 0xbb, 0xa6, - 0x1a, 0x78, 0xa1, 0xeb, 0xf2, 0xf7, 0x15, 0x58, 0xdd, 0x53, 0x41, 0xee, 0x79, 0x52, 0xd9, 0x87, - 0xba, 0xd0, 0xd5, 0x32, 0x8b, 0x2b, 0xba, 0xb9, 0xec, 0x69, 0x65, 0x26, 0x7b, 0x3a, 0x6b, 0x25, - 0x36, 0xa0, 0x32, 0x0e, 0x46, 0x42, 0x48, 0xbb, 0xd9, 0x1d, 0x5a, 0x6c, 0x08, 0xcd, 0xa8, 0x48, - 0x9f, 0x72, 0x43, 0xa1, 0x52, 0xa4, 0x9f, 0x42, 0x8b, 0xa7, 0x77, 0x1c, 0x76, 0x72, 0x28, 0xac, - 0xe2, 0x0a, 0xcf, 0x1e, 0xa8, 0xa5, 0xa3, 0x92, 0x9b, 0xb0, 0x7c, 0x12, 0x04, 0x3f, 0xa0, 0x74, - 0xb6, 0xee, 0xae, 0xe0, 0x94, 0x54, 0xd4, 0x2c, 0x1c, 0x64, 0x21, 0x4a, 0x44, 0x51, 0xd8, 0x86, - 0x61, 0x30, 0xf6, 0x1c, 0xee, 0x17, 0x37, 0xad, 0x8e, 0x80, 0x1e, 0x22, 0x90, 0x7c, 0x09, 0xf5, - 0xf8, 0x2c, 0x76, 0x12, 0xe5, 0x1f, 0xa3, 0xc3, 0x3a, 0xc3, 0xc9, 0xad, 0x01, 0x47, 0x12, 0x69, - 0x5e, 0x31, 0xc5, 0xf8, 0x1c, 0xda, 0xfa, 0xc0, 0x42, 0x27, 0xf6, 0xb7, 0x2d, 0xe8, 0xec, 0xd2, - 0x70, 0x1c, 0x9c, 0x9d, 0x77, 0x5a, 0x1f, 0xcf, 0x64, 0x33, 0x84, 0xfd, 0x9b, 0x59, 0x62, 0x26, - 0xc9, 0x31, 0x3f, 0x0a, 0xb9, 0x0e, 0x4d, 0xe9, 0xbf, 0x49, 0x9d, 0x93, 0x02, 0xd2, 0xbc, 0x75, - 0x55, 0xcf, 0x5b, 0xbf, 0x01, 0x40, 0x5f, 0x24, 0x91, 0x3d, 0x44, 0x7b, 0xcd, 0x83, 0x93, 0x26, - 0x42, 0x98, 0x39, 0x92, 0x21, 0x2c, 0xcf, 0xd3, 0xd7, 0x55, 0x08, 0xfb, 0x8b, 0x5c, 0xaa, 0xbe, - 0x91, 0x89, 0x4b, 0xd6, 0xa0, 0xea, 0x04, 0x53, 0x9f, 0x47, 0xb4, 0x55, 0x8b, 0x77, 0x18, 0x03, - 0xa9, 0x7f, 0x8a, 0x47, 0xd1, 0xb4, 0x58, 0x13, 0x85, 0xce, 0x8f, 0xd1, 0x26, 0x33, 0xa1, 0xe3, - 0xaa, 0x84, 0xaf, 0xe6, 0x24, 0x88, 0x93, 0x18, 0xe3, 0x8c, 0xa6, 0xc5, 0x17, 0xf8, 0x98, 0x41, - 0xf4, 0x88, 0xbf, 0x93, 0x8d, 0xf8, 0xbf, 0xd0, 0xf2, 0x88, 0x3c, 0x82, 0xb8, 0xc1, 0x78, 0x99, - 0x39, 0x86, 0xb9, 0x59, 0xc4, 0x0d, 0x68, 0x89, 0xf6, 0x84, 0xe9, 0x83, 0x15, 0x64, 0x83, 0x0e, - 0x52, 0x0a, 0xbf, 0xa7, 0x29, 0xfc, 0x35, 0xa8, 0xba, 0xf4, 0x68, 0x3a, 0xc2, 0x24, 0x79, 0xc3, - 0xe2, 0x1d, 0x76, 0x0a, 0x41, 0x48, 0xfd, 0x41, 0xe2, 0x7a, 0x3e, 0xa6, 0xc5, 0x1b, 0x56, 0x0a, - 0x20, 0x1f, 0x2b, 0x87, 0xe7, 0x4a, 0x9a, 0xc1, 0xcf, 0x2e, 0xb2, 0xc8, 0xf1, 0xd9, 0x06, 0x60, - 0x27, 0x29, 0xa6, 0xae, 0xa5, 0x11, 0x52, 0x6e, 0x7f, 0x0a, 0x47, 0x86, 0x5f, 0x0a, 0x40, 0x76, - 0x60, 0xc5, 0x45, 0xe4, 0x61, 0x9c, 0x44, 0x76, 0x42, 0x47, 0x67, 0xfd, 0xab, 0x68, 0x45, 0x8c, - 0x59, 0x3a, 0x03, 0x81, 0x61, 0x75, 0xf9, 0x14, 0xd9, 0x27, 0xef, 0xc3, 0xb2, 0x6b, 0x27, 0x76, - 0x7f, 0x1d, 0x57, 0x70, 0x6d, 0x76, 0xe6, 0xae, 0x9d, 0xc8, 0xc8, 0x91, 0x21, 0x32, 0xf9, 0x8a, - 0x83, 0xe3, 0x44, 0x64, 0x09, 0x5e, 0x13, 0xde, 0x66, 0x70, 0x9c, 0xf0, 0x6c, 0xc3, 0x0d, 0x68, - 0xe1, 0x5b, 0xb1, 0x18, 0xef, 0xa3, 0xc0, 0xe0, 0xaa, 0x63, 0x8e, 0x20, 0x2a, 0x4c, 0x8e, 0x3c, - 0xdf, 0xed, 0xbf, 0xce, 0x33, 0xc7, 0x4e, 0x38, 0x7d, 0xe0, 0xf9, 0x2e, 0x16, 0x77, 0x8c, 0x7c, - 0x66, 0x56, 0x50, 0x65, 0x18, 0x5c, 0xa9, 0x71, 0x10, 0x53, 0x1a, 0x64, 0x13, 0xda, 0xdc, 0x30, - 0x39, 0x11, 0xb5, 0x13, 0xda, 0xbf, 0xc6, 0x73, 0x14, 0x08, 0xdb, 0x41, 0x10, 0x23, 0x1f, 0xd9, - 0xcf, 0xb9, 0xf0, 0x5f, 0x47, 0x73, 0x5b, 0x8f, 0xec, 0xe7, 0x28, 0xfa, 0x5a, 0xb8, 0xfa, 0x46, - 0x36, 0x5c, 0x7d, 0x07, 0x56, 0xd5, 0xa5, 0x50, 0x59, 0x92, 0x37, 0xf1, 0x72, 0xac, 0xc8, 0xcb, - 0x31, 0xff, 0x45, 0xe5, 0xc6, 0x9c, 0x17, 0x95, 0xfc, 0x43, 0xc9, 0x46, 0xe1, 0x43, 0x49, 0x41, - 0xb2, 0x67, 0xb3, 0x28, 0xd9, 0xf3, 0x4a, 0x89, 0xee, 0x57, 0x71, 0x2a, 0x59, 0x54, 0x9c, 0x95, - 0xc2, 0x45, 0xfd, 0x60, 0x25, 0x42, 0x17, 0x4d, 0x6c, 0xeb, 0x3a, 0xf7, 0x23, 0x68, 0x28, 0x29, - 0x6d, 0xc0, 0xf2, 0xf6, 0xb7, 0xcf, 0x9e, 0xf6, 0x96, 0x58, 0xeb, 0xe1, 0xfe, 0xc1, 0x41, 0xaf, - 0xc4, 0x5a, 0x7b, 0xdb, 0x3b, 0x8f, 0x7b, 0x65, 0x02, 0x50, 0x7b, 0x74, 0xf0, 0xf4, 0xc1, 0xf6, - 0x41, 0xaf, 0x62, 0xfe, 0x57, 0x19, 0xba, 0x16, 0x0d, 0xc7, 0x76, 0x1a, 0x6c, 0xbe, 0x0f, 0x4d, - 0x57, 0x8a, 0xb4, 0x88, 0x38, 0x57, 0x67, 0xe4, 0xdc, 0x4a, 0x71, 0xc8, 0x5b, 0xd0, 0x15, 0x9a, - 0xc2, 0xf3, 0x4f, 0x68, 0xe4, 0x25, 0x22, 0x7e, 0xcc, 0x41, 0xc9, 0x3e, 0x74, 0x8e, 0xbd, 0x31, - 0x93, 0xc7, 0x4c, 0x44, 0x89, 0xaf, 0x5b, 0xd9, 0x35, 0x6c, 0x3d, 0x44, 0x3c, 0x5d, 0x11, 0xb4, - 0x8f, 0x35, 0x10, 0xb9, 0xc3, 0x1c, 0x97, 0xf0, 0x4c, 0xa4, 0xb2, 0xaf, 0x17, 0x50, 0xd8, 0x09, - 0x42, 0x91, 0x82, 0x41, 0x4c, 0x99, 0x66, 0xad, 0xaa, 0x34, 0xab, 0xf1, 0x35, 0xac, 0xce, 0x7c, - 0x66, 0xd1, 0xa3, 0x52, 0x5f, 0x59, 0xc8, 0x3c, 0x3e, 0x87, 0x55, 0x8c, 0x1c, 0x32, 0x5e, 0xf6, - 0xfc, 0xa0, 0x3c, 0x63, 0xd6, 0xca, 0x79, 0xb3, 0xb6, 0x0e, 0x35, 0xb4, 0x64, 0x9c, 0x9d, 0x4d, - 0x4b, 0xf4, 0x54, 0xba, 0x7a, 0x59, 0x4b, 0x57, 0xff, 0x49, 0x09, 0x08, 0xcf, 0x9a, 0xfc, 0xb6, - 0x3f, 0xcd, 0xb8, 0x11, 0x46, 0x53, 0x5f, 0x06, 0xdc, 0xbc, 0x63, 0x6e, 0x72, 0x16, 0x1e, 0xda, - 0xc9, 0x09, 0x1a, 0xe8, 0x90, 0x35, 0x84, 0x57, 0xcb, 0x3b, 0xe6, 0x9f, 0x95, 0x98, 0xe7, 0x18, - 0x2a, 0x4f, 0xe2, 0x1e, 0xd4, 0x79, 0xe1, 0xa1, 0x7c, 0x35, 0xb8, 0xce, 0x5f, 0x0d, 0x14, 0xc6, - 0xd6, 0x33, 0x3e, 0x2c, 0xfc, 0x19, 0x81, 0x6c, 0xec, 0x43, 0x5b, 0x1f, 0x28, 0x38, 0xb0, 0x9b, - 0xd9, 0x10, 0xa9, 0x23, 0xe9, 0xe2, 0xea, 0xf4, 0xf3, 0xfb, 0x65, 0x09, 0x5a, 0x03, 0xea, 0xbb, - 0xf3, 0x53, 0xf8, 0xef, 0x09, 0x33, 0x51, 0x4e, 0x9f, 0xe2, 0xb5, 0x09, 0x79, 0x23, 0xf1, 0xf2, - 0x97, 0xfe, 0x0b, 0x68, 0xed, 0x45, 0x51, 0x10, 0xf1, 0x3a, 0x62, 0xe5, 0xcf, 0x97, 0x50, 0x23, - 0x62, 0x9b, 0x1d, 0xee, 0x84, 0x3f, 0xae, 0x49, 0x9f, 0x58, 0x74, 0xcd, 0x7f, 0x2e, 0xe9, 0xd1, - 0xde, 0xbc, 0x17, 0xb8, 0x75, 0xf5, 0xb6, 0x25, 0xca, 0x3e, 0xc5, 0xfb, 0x99, 0x01, 0x8d, 0x30, - 0x0a, 0x46, 0x11, 0x8d, 0x63, 0xf9, 0xd6, 0x24, 0xfb, 0xf3, 0xdf, 0xd6, 0x62, 0x7c, 0x60, 0x92, - 0xbf, 0xaa, 0xe0, 0x3d, 0x72, 0x17, 0xda, 0x88, 0x30, 0xe4, 0xa5, 0xba, 0xe8, 0x84, 0x09, 0xdf, - 0x57, 0xdb, 0x9c, 0xd5, 0xa2, 0x69, 0xc7, 0x8c, 0xa1, 0x26, 0xaa, 0x66, 0xb6, 0x54, 0xf5, 0x5d, - 0x29, 0x8d, 0x94, 0xf9, 0x58, 0x51, 0xfd, 0xdd, 0xab, 0x14, 0x7e, 0xfd, 0x5d, 0x1d, 0xd6, 0xb9, - 0xdd, 0x54, 0x4f, 0x3d, 0x92, 0x6b, 0xf3, 0xaf, 0x90, 0x9e, 0x64, 0x2c, 0xe7, 0x92, 0x8c, 0x9c, - 0xd7, 0x15, 0xc5, 0xeb, 0xa2, 0x7a, 0x12, 0xc5, 0xcb, 0xaa, 0xce, 0x4b, 0xac, 0x9f, 0x75, 0x1c, - 0xc6, 0xfc, 0x1a, 0xf7, 0x0a, 0x44, 0x97, 0x7c, 0x2c, 0x93, 0xea, 0xaa, 0x08, 0xa9, 0x78, 0xc9, - 0xf3, 0x8a, 0x51, 0x1a, 0xc5, 0xc5, 0x28, 0xd9, 0xcc, 0xfb, 0x76, 0xbe, 0x72, 0xe4, 0xd6, 0x39, - 0x1f, 0x2a, 0x2e, 0x23, 0x21, 0x22, 0xd2, 0x69, 0xa1, 0x4c, 0xf3, 0xc0, 0x66, 0x7e, 0x11, 0xc9, - 0xcf, 0xb3, 0xd5, 0x1f, 0xbc, 0xd0, 0xf4, 0x9d, 0x73, 0x3e, 0xba, 0x50, 0x29, 0x48, 0xf7, 0x52, - 0xa5, 0x20, 0x2b, 0x97, 0x2c, 0x05, 0xe9, 0x15, 0x3e, 0x3e, 0xd9, 0xc5, 0x15, 0x1e, 0xab, 0xb8, - 0x93, 0x0f, 0x2e, 0xb5, 0x93, 0xdf, 0x54, 0xb9, 0xc7, 0xff, 0xdb, 0x9a, 0x8d, 0xbf, 0x28, 0xc1, - 0x6b, 0xc2, 0x2f, 0x98, 0xb9, 0xb1, 0x77, 0xa1, 0x26, 0xdc, 0x62, 0xee, 0xe3, 0x18, 0xf3, 0x8f, - 0xc0, 0x12, 0x98, 0x6c, 0x4e, 0x84, 0xe6, 0x53, 0x7c, 0xd6, 0xe0, 0x8e, 0x47, 0xe6, 0x59, 0x59, - 0xcd, 0xe1, 0x98, 0xe9, 0xfd, 0xad, 0x68, 0xf7, 0xd7, 0x3c, 0xd3, 0x5d, 0x00, 0xb9, 0x24, 0x15, - 0xa1, 0x96, 0xf2, 0x95, 0x55, 0xe2, 0xaa, 0x97, 0xb3, 0x57, 0xfd, 0xbc, 0xe7, 0x7e, 0x4d, 0xed, - 0x2f, 0x67, 0xd5, 0xfe, 0x1f, 0x65, 0x7c, 0x80, 0x57, 0xf8, 0xb6, 0x20, 0x28, 0xad, 0xbf, 0xea, - 0x9b, 0xdf, 0xcd, 0xbc, 0xb5, 0xcf, 0x33, 0x2e, 0xf3, 0xe9, 0x4b, 0xf5, 0x20, 0x7e, 0xcb, 0xc0, - 0xda, 0xe6, 0x83, 0xc2, 0xd7, 0xf6, 0x79, 0xb4, 0x15, 0xe3, 0xcb, 0x3a, 0xe3, 0xef, 0xb3, 0xb5, - 0xe1, 0x03, 0xbc, 0x7c, 0x64, 0x59, 0x6c, 0xfe, 0x2f, 0x85, 0x3b, 0xb2, 0xa8, 0xb9, 0x94, 0xaa, - 0xbd, 0x92, 0x4d, 0xa2, 0x32, 0x1f, 0x47, 0xaa, 0x7b, 0xd6, 0x9e, 0xa3, 0xee, 0x89, 0xf0, 0x27, - 0x78, 0x46, 0x14, 0xdb, 0xe6, 0x23, 0xee, 0x84, 0xcc, 0x5b, 0x88, 0x24, 0x5e, 0x2e, 0x22, 0x9e, - 0x91, 0xc5, 0xa7, 0xb0, 0xbe, 0x9d, 0x24, 0xb6, 0x73, 0x32, 0xc3, 0xd2, 0x4d, 0x68, 0xab, 0x1a, - 0x8c, 0xa1, 0xa2, 0xde, 0x52, 0xb0, 0x7d, 0x57, 0xad, 0xac, 0xac, 0xad, 0xec, 0xaf, 0x4a, 0xb0, - 0x6a, 0x4d, 0xfd, 0x6d, 0xdf, 0xfd, 0xde, 0xf6, 0x54, 0xc2, 0xee, 0x53, 0x10, 0xc1, 0xf4, 0x30, - 0xe0, 0x90, 0xf9, 0xc1, 0x45, 0xc7, 0xcd, 0x24, 0x8f, 0x7a, 0x50, 0x71, 0x26, 0xae, 0xf8, 0x04, - 0x6b, 0xb2, 0x8d, 0xd8, 0xf1, 0x99, 0xef, 0xc8, 0x84, 0x29, 0x76, 0x98, 0x5e, 0xc7, 0xc6, 0x30, - 0xf1, 0x26, 0x34, 0x98, 0x26, 0xc2, 0x01, 0x6d, 0x23, 0xf0, 0x19, 0x87, 0x99, 0xdf, 0xc2, 0x6b, - 0x6c, 0x9f, 0x51, 0x30, 0xbe, 0x44, 0x25, 0x88, 0xcc, 0x7e, 0x96, 0xb5, 0xec, 0x67, 0x71, 0xb2, - 0x76, 0x30, 0x4b, 0x76, 0x21, 0xc1, 0xca, 0x08, 0xbc, 0xb0, 0x87, 0xe6, 0x11, 0xf4, 0x0e, 0x82, - 0x51, 0xb6, 0x3a, 0xa8, 0xe0, 0x9c, 0xd1, 0x6b, 0x92, 0x4b, 0x64, 0x7e, 0xe0, 0x1a, 0x54, 0x63, - 0xcf, 0x77, 0xa4, 0xb4, 0xf1, 0x0e, 0x83, 0x4e, 0xfd, 0xc4, 0x1b, 0x4b, 0xaf, 0x0c, 0x3b, 0xe6, - 0x81, 0xf6, 0x8d, 0x85, 0x57, 0x8c, 0x47, 0x5f, 0xd1, 0x8e, 0xfe, 0x5f, 0x4b, 0xf0, 0xda, 0xde, - 0x0b, 0xea, 0x4c, 0x0b, 0xca, 0x61, 0x2e, 0x21, 0x4d, 0xfa, 0xe3, 0x5b, 0x39, 0xf7, 0xf8, 0x46, - 0xc4, 0xe3, 0x1b, 0xd7, 0x36, 0xfc, 0x9d, 0xad, 0x0f, 0x75, 0x16, 0x61, 0xa6, 0xa9, 0x5b, 0xd9, - 0x25, 0x6f, 0x00, 0x04, 0x21, 0xf5, 0x87, 0x31, 0xa6, 0xa1, 0xaa, 0xf9, 0x34, 0xd4, 0xeb, 0xd0, - 0x88, 0x68, 0x38, 0x1e, 0x32, 0xb9, 0xaa, 0x89, 0xbc, 0x07, 0x0d, 0xc7, 0x3b, 0x13, 0xf7, 0x9d, - 0x5b, 0x50, 0xe3, 0x05, 0x2f, 0x2c, 0x64, 0xfe, 0xf9, 0xde, 0xde, 0x21, 0x0f, 0xa3, 0x9f, 0x59, - 0xdf, 0xee, 0xf1, 0x67, 0x80, 0x87, 0xdb, 0x07, 0x83, 0xbd, 0x5e, 0xf9, 0xee, 0x3f, 0xad, 0x42, - 0x7d, 0x27, 0x88, 0xa8, 0x75, 0xb8, 0x43, 0x36, 0x61, 0x19, 0x7f, 0xb0, 0x8c, 0xef, 0x2a, 0xf8, - 0x43, 0x66, 0xa3, 0xcd, 0xe3, 0x07, 0xfe, 0x4b, 0x66, 0x73, 0x89, 0xdc, 0x03, 0xf2, 0xbd, 0x9d, - 0x38, 0x27, 0xd9, 0x5f, 0x10, 0x6b, 0x13, 0x56, 0x79, 0x98, 0xa0, 0x8d, 0x9a, 0x4b, 0x77, 0x4a, - 0xe4, 0x1e, 0xb4, 0xb5, 0x1f, 0x9a, 0xc5, 0x64, 0x5d, 0xbd, 0xc0, 0x66, 0x7e, 0xb9, 0xc5, 0xbf, - 0x27, 0xb1, 0xcc, 0x25, 0xf2, 0x39, 0x74, 0xb3, 0x3f, 0x2c, 0x23, 0xb2, 0x24, 0x78, 0xf6, 0x67, - 0x5f, 0x86, 0xfe, 0xbb, 0x2a, 0x73, 0x89, 0xdc, 0x87, 0xd5, 0x99, 0x5f, 0x88, 0x11, 0x0c, 0xb4, - 0xe6, 0xfd, 0x70, 0xcc, 0x48, 0x37, 0x62, 0x2e, 0x91, 0xb7, 0xa0, 0xc6, 0x4b, 0x5f, 0xc8, 0xaa, - 0x28, 0x63, 0x48, 0xab, 0x54, 0x0c, 0xf9, 0x1b, 0x5d, 0x73, 0x89, 0x6c, 0x41, 0x53, 0x55, 0xba, - 0x90, 0xb5, 0xd4, 0x9a, 0x6a, 0xd8, 0x79, 0xba, 0xbc, 0xe2, 0x85, 0xd3, 0xcd, 0x54, 0xbf, 0xe8, - 0x74, 0x37, 0xa1, 0xc1, 0x38, 0x84, 0x3f, 0x10, 0xd6, 0x38, 0xdc, 0x10, 0x18, 0x31, 0x1e, 0x47, - 0x97, 0x4f, 0x57, 0xbf, 0x5d, 0x2c, 0x20, 0xb9, 0x92, 0xfb, 0x71, 0xaa, 0xb9, 0x44, 0xde, 0x86, - 0xba, 0xa8, 0xca, 0x20, 0x64, 0xb6, 0x44, 0xc3, 0x50, 0x3f, 0x02, 0x31, 0x97, 0xc8, 0x1d, 0x80, - 0xb4, 0x46, 0x81, 0x5c, 0x4d, 0xb7, 0xa7, 0x4f, 0xc8, 0xec, 0xef, 0x03, 0x7e, 0xd6, 0x87, 0x81, - 0xcb, 0x7f, 0x61, 0xb2, 0x56, 0xf4, 0xda, 0xce, 0xa7, 0x20, 0x84, 0xaf, 0x47, 0xd4, 0x02, 0x90, - 0x82, 0xc2, 0x80, 0xcc, 0x7a, 0xde, 0x86, 0xfa, 0x40, 0x47, 0x1d, 0xcc, 0x47, 0xdd, 0x86, 0x95, - 0x5c, 0x2d, 0x02, 0x31, 0x34, 0xea, 0xb9, 0x02, 0x05, 0xa3, 0xa7, 0x7e, 0xfa, 0x92, 0x32, 0xea, - 0x0e, 0xb4, 0x1f, 0x69, 0x25, 0x93, 0x64, 0x25, 0x53, 0xdd, 0xb7, 0xbf, 0x6b, 0x64, 0xcb, 0xfd, - 0xcc, 0x25, 0xf2, 0x21, 0xd6, 0x33, 0x69, 0x45, 0x82, 0xbd, 0xdc, 0x94, 0xd8, 0xe8, 0x66, 0x20, - 0x31, 0x8a, 0x6a, 0x37, 0xfb, 0xa3, 0x75, 0x2e, 0xe6, 0x85, 0x3f, 0x64, 0x9f, 0xf9, 0xe4, 0x9d, - 0x12, 0xf9, 0x5c, 0xfc, 0x8e, 0x33, 0x70, 0xa9, 0x46, 0xa3, 0x88, 0x95, 0xb3, 0xdf, 0xfe, 0x1a, - 0xae, 0x3c, 0x9a, 0xad, 0x0a, 0x2d, 0x58, 0xf6, 0x5a, 0x76, 0xaa, 0xbc, 0xdd, 0xe4, 0x09, 0x5c, - 0x29, 0x28, 0x2b, 0x25, 0xf2, 0x97, 0x27, 0x73, 0xea, 0x4d, 0xe7, 0x92, 0x1b, 0xe2, 0x6f, 0x49, - 0x67, 0x2b, 0x3a, 0xc9, 0xc6, 0x45, 0xc5, 0x9e, 0xc6, 0x7c, 0x0c, 0x61, 0x14, 0x90, 0x59, 0xef, - 0xc2, 0x32, 0x73, 0x7e, 0xe4, 0x59, 0xaa, 0x9c, 0x8b, 0xa1, 0x00, 0x39, 0x6c, 0xe6, 0xa1, 0x70, - 0x6c, 0x2d, 0xff, 0x61, 0x28, 0x80, 0x8e, 0x7d, 0x1f, 0x20, 0xcd, 0x46, 0x90, 0xab, 0x85, 0x4f, - 0xbb, 0x46, 0x0e, 0x9c, 0x9b, 0x9f, 0xba, 0xd4, 0x7c, 0xfe, 0x4c, 0x96, 0xcd, 0xc8, 0x81, 0xf5, - 0xf9, 0xdb, 0xd0, 0xd2, 0xfc, 0x62, 0xae, 0x66, 0x67, 0x93, 0x65, 0x46, 0x1e, 0xae, 0x93, 0xd8, - 0x85, 0x95, 0x5c, 0x04, 0x41, 0x66, 0xbd, 0x1b, 0xe3, 0x9c, 0x48, 0x03, 0xa9, 0x3c, 0x82, 0x5e, - 0x3e, 0x68, 0xe1, 0xf2, 0x98, 0x4d, 0x71, 0x1a, 0xd7, 0x34, 0x58, 0x21, 0xa1, 0x27, 0xb0, 0x92, - 0xf3, 0xc3, 0x49, 0x51, 0xc4, 0x92, 0x59, 0x57, 0xb1, 0xe3, 0x8e, 0xe4, 0xfe, 0x00, 0xae, 0x14, - 0xb8, 0xdf, 0x5c, 0x58, 0xe7, 0x57, 0xc1, 0x1a, 0xf3, 0xc6, 0x75, 0xd2, 0x87, 0xbc, 0xb0, 0x5c, - 0xf7, 0x9e, 0xc8, 0x35, 0x29, 0x91, 0x05, 0xae, 0x9a, 0x51, 0x38, 0xa8, 0x53, 0xfc, 0x05, 0xf4, - 0xf2, 0x7e, 0x08, 0xa7, 0x38, 0xc7, 0x3b, 0xe1, 0xbb, 0x2f, 0xf6, 0x83, 0xcd, 0xa5, 0xdb, 0xa5, - 0x3b, 0x25, 0xb2, 0xc7, 0xd8, 0x99, 0x09, 0x1d, 0xe4, 0xb1, 0xe8, 0x05, 0xbd, 0x92, 0x8d, 0x45, - 0x31, 0x06, 0xae, 0xec, 0x0b, 0x68, 0x2a, 0x87, 0x4b, 0x28, 0xf8, 0x9c, 0x8f, 0x67, 0x64, 0xa1, - 0xfa, 0xe4, 0x3d, 0x80, 0xd4, 0xb3, 0x16, 0x26, 0x25, 0xef, 0x69, 0x5f, 0xbc, 0x95, 0xa3, 0x1a, - 0xfe, 0xe3, 0x95, 0x0f, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0xc1, 0x41, 0xa0, 0x83, 0x86, 0x45, - 0x00, 0x00, +var File_core_proto protoreflect.FileDescriptor + +var file_core_proto_rawDesc = []byte{ + 0x0a, 0x0a, 0x63, 0x6f, 0x72, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x02, 0x70, 0x62, + 0x22, 0x07, 0x0a, 0x05, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x99, 0x01, 0x0a, 0x08, 0x43, 0x6f, + 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x76, 0x69, 0x73, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x72, 0x65, 0x76, 0x69, 0x73, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x75, + 0x69, 0x6c, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x75, + 0x69, 0x6c, 0x64, 0x41, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x67, 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x5f, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x67, + 0x6f, 0x6c, 0x61, 0x6e, 0x67, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x17, 0x0a, 0x07, + 0x6f, 0x73, 0x5f, 0x61, 0x72, 0x63, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6f, + 0x73, 0x41, 0x72, 0x63, 0x68, 0x22, 0x5b, 0x0a, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, + 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x61, 0x64, 0x64, 0x72, 0x65, + 0x73, 0x73, 0x65, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, + 0x5f, 0x69, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x10, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x49, 0x6e, 0x53, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x22, 0xfd, 0x01, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x61, + 0x69, 0x6e, 0x65, 0x72, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x18, 0x0a, 0x07, + 0x61, 0x70, 0x70, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, + 0x70, 0x70, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, + 0x6d, 0x65, 0x12, 0x3d, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x74, + 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4c, 0x61, + 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, + 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x2d, 0x0a, 0x03, 0x50, 0x6f, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x64, 0x65, 0x73, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x64, 0x65, 0x73, + 0x63, 0x22, 0x23, 0x0a, 0x04, 0x50, 0x6f, 0x64, 0x73, 0x12, 0x1b, 0x0a, 0x04, 0x70, 0x6f, 0x64, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x07, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x64, + 0x52, 0x04, 0x70, 0x6f, 0x64, 0x73, 0x22, 0xdd, 0x06, 0x0a, 0x0b, 0x50, 0x6f, 0x64, 0x52, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x0c, 0x63, 0x70, + 0x75, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x20, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x2e, 0x43, 0x70, 0x75, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x0b, 0x63, 0x70, 0x75, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, 0x12, + 0x4c, 0x0a, 0x0f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, + 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, + 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, + 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x6d, + 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x48, 0x0a, + 0x0d, 0x76, 0x65, 0x72, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x64, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x56, 0x65, 0x72, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x76, 0x65, 0x72, 0x69, 0x66, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x36, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, + 0x6c, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, + 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x12, + 0x4f, 0x0a, 0x10, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x62, 0x2e, 0x50, + 0x6f, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, + 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, + 0x12, 0x4c, 0x0a, 0x0f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x62, 0x2e, 0x50, + 0x6f, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, + 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x3e, + 0x0a, 0x10, 0x43, 0x70, 0x75, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x41, + 0x0a, 0x13, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x1a, 0x40, 0x0a, 0x12, 0x56, 0x65, 0x72, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x1a, 0x3a, 0x0a, 0x0c, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, + 0x42, 0x0a, 0x14, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, + 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x1a, 0x41, 0x0a, 0x13, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x50, 0x65, 0x72, + 0x63, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf8, 0x01, 0x0a, 0x0c, 0x4e, 0x6f, 0x64, 0x65, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, + 0x70, 0x75, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, + 0x52, 0x0a, 0x63, 0x70, 0x75, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x25, 0x0a, 0x0e, + 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x01, 0x52, 0x0d, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x50, 0x65, 0x72, 0x63, + 0x65, 0x6e, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x76, 0x65, 0x72, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x76, 0x65, 0x72, 0x69, 0x66, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, + 0x6c, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x65, 0x72, + 0x63, 0x65, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x72, + 0x61, 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x76, 0x6f, + 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x01, 0x52, 0x0d, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, + 0x74, 0x22, 0x46, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x16, 0x0a, 0x06, 0x64, 0x72, 0x69, 0x76, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x64, 0x72, 0x69, 0x76, 0x65, 0x72, 0x22, 0x71, 0x0a, 0x15, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x12, 0x16, 0x0a, 0x06, + 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x69, 0x70, 0x76, 0x34, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x69, 0x70, 0x76, 0x34, 0x12, 0x12, 0x0a, 0x04, 0x69, 0x70, 0x76, 0x36, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x69, 0x70, 0x76, 0x36, 0x22, 0x62, 0x0a, 0x18, + 0x44, 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, + 0x6b, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6e, 0x65, 0x74, 0x77, + 0x6f, 0x72, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6e, 0x65, 0x74, 0x77, 0x6f, + 0x72, 0x6b, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, + 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, + 0x22, 0x37, 0x0a, 0x07, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, + 0x18, 0x0a, 0x07, 0x73, 0x75, 0x62, 0x6e, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x07, 0x73, 0x75, 0x62, 0x6e, 0x65, 0x74, 0x73, 0x22, 0x33, 0x0a, 0x08, 0x4e, 0x65, 0x74, + 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x27, 0x0a, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x65, 0x74, + 0x77, 0x6f, 0x72, 0x6b, 0x52, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x22, 0xea, + 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x65, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x23, 0x0a, 0x03, 0x63, 0x70, 0x75, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x11, + 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x2e, 0x43, 0x70, 0x75, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x03, 0x63, 0x70, 0x75, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x70, 0x75, 0x5f, 0x75, 0x73, + 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, 0x52, 0x07, 0x63, 0x70, 0x75, 0x55, 0x73, 0x65, + 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x06, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x65, 0x6d, + 0x6f, 0x72, 0x79, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, + 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x55, 0x73, 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x61, 0x76, + 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, + 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, + 0x6c, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, + 0x64, 0x65, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, + 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x6e, 0x69, 0x74, 0x5f, 0x6d, + 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x69, 0x6e, 0x69, + 0x74, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x30, 0x0a, 0x08, 0x69, 0x6e, 0x69, 0x74, 0x5f, + 0x63, 0x70, 0x75, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x4e, + 0x6f, 0x64, 0x65, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x43, 0x70, 0x75, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x07, 0x69, 0x6e, 0x69, 0x74, 0x43, 0x70, 0x75, 0x12, 0x12, 0x0a, 0x04, 0x69, 0x6e, 0x66, + 0x6f, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x12, 0x26, 0x0a, + 0x04, 0x6e, 0x75, 0x6d, 0x61, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x62, + 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x2e, 0x4e, 0x75, 0x6d, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x04, 0x6e, 0x75, 0x6d, 0x61, 0x12, 0x39, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x61, 0x5f, 0x6d, 0x65, + 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x0e, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x62, 0x2e, + 0x4e, 0x6f, 0x64, 0x65, 0x2e, 0x4e, 0x75, 0x6d, 0x61, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x6e, 0x75, 0x6d, 0x61, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, + 0x12, 0x18, 0x0a, 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x18, 0x0f, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, + 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x10, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x55, 0x73, 0x65, 0x64, 0x12, 0x21, 0x0a, + 0x0c, 0x69, 0x6e, 0x69, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x18, 0x11, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0b, 0x69, 0x6e, 0x69, 0x74, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, + 0x12, 0x39, 0x0a, 0x0b, 0x69, 0x6e, 0x69, 0x74, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x18, + 0x12, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x2e, + 0x49, 0x6e, 0x69, 0x74, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x0a, 0x69, 0x6e, 0x69, 0x74, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x76, + 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x18, 0x13, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x62, + 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x06, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x76, 0x6f, 0x6c, + 0x75, 0x6d, 0x65, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x14, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, + 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x55, 0x73, 0x65, 0x64, 0x1a, 0x36, 0x0a, 0x08, 0x43, 0x70, + 0x75, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3a, 0x0a, + 0x0c, 0x49, 0x6e, 0x69, 0x74, 0x43, 0x70, 0x75, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x37, 0x0a, 0x09, 0x4e, 0x75, 0x6d, + 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x1a, 0x3d, 0x0a, 0x0f, 0x4e, 0x75, 0x6d, 0x61, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x1a, 0x3d, 0x0a, 0x0f, 0x49, 0x6e, 0x69, 0x74, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x1a, 0x39, 0x0a, 0x0b, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x27, 0x0a, 0x05, 0x4e, + 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x08, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x6e, + 0x6f, 0x64, 0x65, 0x73, 0x22, 0x45, 0x0a, 0x0d, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x76, 0x61, 0x69, + 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0xc3, 0x06, 0x0a, 0x0e, + 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1a, + 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x29, 0x0a, 0x0a, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x5f, 0x6f, 0x70, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, + 0x2e, 0x70, 0x62, 0x2e, 0x54, 0x72, 0x69, 0x4f, 0x70, 0x74, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x4f, 0x70, 0x74, 0x12, 0x3d, 0x0a, 0x09, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x63, + 0x70, 0x75, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x65, + 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x44, 0x65, 0x6c, + 0x74, 0x61, 0x43, 0x70, 0x75, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x64, 0x65, 0x6c, 0x74, + 0x61, 0x43, 0x70, 0x75, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x6d, 0x65, + 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x64, 0x65, 0x6c, 0x74, + 0x61, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x64, 0x65, 0x6c, 0x74, 0x61, + 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x64, 0x65, 0x6c, 0x74, 0x61, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x12, 0x53, 0x0a, 0x11, + 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x61, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, + 0x79, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x74, + 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x44, 0x65, 0x6c, 0x74, + 0x61, 0x4e, 0x75, 0x6d, 0x61, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x0f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x4e, 0x75, 0x6d, 0x61, 0x4d, 0x65, 0x6d, 0x6f, 0x72, + 0x79, 0x12, 0x30, 0x0a, 0x04, 0x6e, 0x75, 0x6d, 0x61, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x2e, 0x4e, 0x75, 0x6d, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x6e, + 0x75, 0x6d, 0x61, 0x12, 0x36, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x08, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x46, 0x0a, 0x0c, 0x64, + 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x18, 0x09, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x56, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x56, 0x6f, 0x6c, + 0x75, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, + 0x73, 0x5f, 0x64, 0x6f, 0x77, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x63, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, 0x44, 0x6f, 0x77, 0x6e, 0x1a, 0x3b, 0x0a, 0x0d, + 0x44, 0x65, 0x6c, 0x74, 0x61, 0x43, 0x70, 0x75, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x44, 0x65, 0x6c, + 0x74, 0x61, 0x4e, 0x75, 0x6d, 0x61, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x37, 0x0a, + 0x09, 0x4e, 0x75, 0x6d, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x1a, 0x3e, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0xd2, 0x03, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x12, + 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, + 0x18, 0x0a, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x6f, 0x64, + 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, + 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x69, + 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x70, + 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x64, 0x12, 0x31, 0x0a, 0x06, 0x6c, 0x61, 0x62, + 0x65, 0x6c, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x43, + 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x34, 0x0a, 0x07, + 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x2e, 0x50, 0x75, 0x62, + 0x6c, 0x69, 0x73, 0x68, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x70, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x28, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x1a, + 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3a, 0x0a, 0x0c, 0x50, 0x75, + 0x62, 0x6c, 0x69, 0x73, 0x68, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x81, 0x02, 0x0a, 0x0f, 0x43, 0x6f, 0x6e, 0x74, 0x61, + 0x69, 0x6e, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x75, + 0x6e, 0x6e, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x72, 0x75, 0x6e, + 0x6e, 0x69, 0x6e, 0x67, 0x12, 0x18, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x79, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x79, 0x12, 0x3d, + 0x0a, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x21, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x1c, 0x0a, + 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x74, + 0x74, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x74, 0x74, 0x6c, 0x1a, 0x3b, 0x0a, + 0x0d, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3f, 0x0a, 0x10, 0x43, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2b, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, + 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0xb6, 0x01, 0x0a, 0x1c, + 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x0e, 0x0a, 0x02, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2b, 0x0a, 0x09, + 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x52, 0x09, + 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x12, 0x2b, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x43, + 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x16, 0x0a, 0x06, + 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x22, 0x49, 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x61, + 0x69, 0x6e, 0x65, 0x72, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x2b, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, + 0xf5, 0x01, 0x0a, 0x1c, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x18, 0x0a, 0x07, 0x61, 0x70, 0x70, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x65, 0x6e, + 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x6f, + 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, + 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, + 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, + 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x1a, 0x39, 0x0a, 0x0b, + 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x0a, 0x43, 0x6f, 0x6e, 0x74, 0x61, + 0x69, 0x6e, 0x65, 0x72, 0x73, 0x12, 0x2d, 0x0a, 0x0a, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, + 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x43, + 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, + 0x6e, 0x65, 0x72, 0x73, 0x22, 0x1d, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, + 0x72, 0x49, 0x44, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x02, 0x69, 0x64, 0x22, 0x20, 0x0a, 0x0c, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, + 0x49, 0x44, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x03, 0x69, 0x64, 0x73, 0x22, 0x54, 0x0a, 0x16, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x43, + 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, + 0x73, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x74, 0x65, 0x70, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x73, 0x74, 0x65, 0x70, 0x22, 0x2e, 0x0a, 0x1a, 0x44, + 0x69, 0x73, 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, + 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x73, 0x22, 0x88, 0x01, 0x0a, 0x0e, + 0x52, 0x65, 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x0e, + 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2c, + 0x0a, 0x0c, 0x62, 0x69, 0x6e, 0x64, 0x5f, 0x63, 0x70, 0x75, 0x5f, 0x6f, 0x70, 0x74, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x54, 0x72, 0x69, 0x4f, 0x70, 0x74, + 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x64, 0x43, 0x70, 0x75, 0x4f, 0x70, 0x74, 0x12, 0x38, 0x0a, 0x0d, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x4f, 0x70, 0x74, 0x73, 0x22, 0x37, 0x0a, 0x0d, 0x41, 0x64, 0x64, 0x50, 0x6f, 0x64, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, + 0x65, 0x73, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x64, 0x65, 0x73, 0x63, 0x22, + 0x26, 0x0a, 0x10, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x50, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x23, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x50, 0x6f, + 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0xd4, 0x05, 0x0a, + 0x0e, 0x41, 0x64, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x1a, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x65, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x63, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x63, + 0x61, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x72, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x63, 0x65, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x63, 0x70, 0x75, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x63, 0x70, 0x75, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, + 0x72, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x65, 0x12, + 0x16, 0x0a, 0x06, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x06, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x36, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, + 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x64, 0x64, + 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4c, 0x61, 0x62, 0x65, + 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, + 0x30, 0x0a, 0x04, 0x6e, 0x75, 0x6d, 0x61, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x70, 0x62, 0x2e, 0x41, 0x64, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x2e, 0x4e, 0x75, 0x6d, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x6e, 0x75, 0x6d, + 0x61, 0x12, 0x43, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x61, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, + 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x64, 0x64, 0x4e, + 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4e, 0x75, 0x6d, 0x61, 0x4d, + 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x6e, 0x75, 0x6d, 0x61, + 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, + 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, + 0x12, 0x40, 0x0a, 0x0a, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x18, 0x0e, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x64, 0x64, 0x4e, 0x6f, 0x64, + 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, + 0x61, 0x70, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, + 0x61, 0x70, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x37, 0x0a, + 0x09, 0x4e, 0x75, 0x6d, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3d, 0x0a, 0x0f, 0x4e, 0x75, 0x6d, 0x61, 0x4d, 0x65, + 0x6d, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3c, 0x0a, 0x0e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x4d, + 0x61, 0x70, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x2f, 0x0a, 0x11, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4e, 0x6f, 0x64, + 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, + 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x9f, 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, + 0x61, 0x6d, 0x65, 0x12, 0x36, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, + 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x52, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, + 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x26, 0x0a, 0x04, 0x6f, 0x70, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, + 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x52, 0x04, 0x6f, 0x70, 0x74, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x66, 0x69, 0x78, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x66, 0x69, 0x78, 0x22, 0xb3, 0x01, 0x0a, 0x10, 0x4c, + 0x69, 0x73, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x18, 0x0a, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x61, 0x6c, 0x6c, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x61, 0x6c, 0x6c, 0x12, 0x38, 0x0a, 0x06, 0x6c, + 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x62, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, + 0x61, 0x62, 0x65, 0x6c, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0xdc, 0x05, 0x0a, 0x05, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x12, + 0x0a, 0x04, 0x72, 0x65, 0x70, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x65, + 0x70, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, + 0x64, 0x69, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x64, 0x69, 0x72, 0x12, 0x1c, + 0x0a, 0x09, 0x73, 0x75, 0x62, 0x6d, 0x6f, 0x64, 0x75, 0x6c, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x09, 0x73, 0x75, 0x62, 0x6d, 0x6f, 0x64, 0x75, 0x6c, 0x65, 0x12, 0x1a, 0x0a, 0x08, + 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, + 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x12, 0x27, 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, + 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x42, 0x75, 0x69, 0x6c, + 0x64, 0x2e, 0x45, 0x6e, 0x76, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x65, 0x6e, 0x76, + 0x73, 0x12, 0x27, 0x0a, 0x04, 0x61, 0x72, 0x67, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x2e, 0x41, 0x72, 0x67, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x61, 0x72, 0x67, 0x73, 0x12, 0x2d, 0x0a, 0x06, 0x6c, 0x61, + 0x62, 0x65, 0x6c, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x70, 0x62, 0x2e, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x36, 0x0a, 0x09, 0x61, 0x72, 0x74, + 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, + 0x62, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, + 0x73, 0x12, 0x2a, 0x0a, 0x05, 0x63, 0x61, 0x63, 0x68, 0x65, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x2e, 0x43, 0x61, 0x63, 0x68, + 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x63, 0x61, 0x63, 0x68, 0x65, 0x12, 0x1f, 0x0a, + 0x0b, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x18, 0x0c, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x73, 0x74, 0x6f, 0x70, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x6c, 0x12, 0x1a, + 0x0a, 0x08, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x08, 0x73, 0x65, 0x63, 0x75, 0x72, 0x69, 0x74, 0x79, 0x1a, 0x37, 0x0a, 0x09, 0x45, 0x6e, + 0x76, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x1a, 0x37, 0x0a, 0x09, 0x41, 0x72, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x39, 0x0a, 0x0b, + 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3c, 0x0a, 0x0e, 0x41, 0x72, 0x74, 0x69, 0x66, + 0x61, 0x63, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x38, 0x0a, 0x0a, 0x43, 0x61, 0x63, 0x68, 0x65, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0x96, 0x01, 0x0a, 0x06, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, + 0x61, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x67, + 0x65, 0x73, 0x12, 0x2e, 0x0a, 0x06, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x70, 0x62, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x73, 0x2e, 0x42, + 0x75, 0x69, 0x6c, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x62, 0x75, 0x69, 0x6c, + 0x64, 0x73, 0x1a, 0x44, 0x0a, 0x0b, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x1f, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xa4, 0x02, 0x0a, 0x11, 0x42, 0x75, 0x69, + 0x6c, 0x64, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, + 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x03, 0x75, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, + 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x12, 0x22, 0x0a, 0x06, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x70, + 0x62, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x73, 0x52, 0x06, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x73, + 0x12, 0x10, 0x0a, 0x03, 0x74, 0x61, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x74, + 0x61, 0x72, 0x12, 0x44, 0x0a, 0x0c, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x68, + 0x6f, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x70, 0x62, 0x2e, 0x42, 0x75, + 0x69, 0x6c, 0x64, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x52, 0x0b, 0x62, 0x75, 0x69, + 0x6c, 0x64, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x78, 0x69, 0x73, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x65, 0x78, 0x69, 0x73, + 0x74, 0x49, 0x64, 0x22, 0x2a, 0x0a, 0x0b, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x4d, 0x65, 0x74, 0x68, + 0x6f, 0x64, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x43, 0x4d, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x52, + 0x41, 0x57, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x58, 0x49, 0x53, 0x54, 0x10, 0x02, 0x22, + 0x65, 0x0a, 0x0b, 0x48, 0x6f, 0x6f, 0x6b, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1f, + 0x0a, 0x0b, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x66, 0x74, 0x65, 0x72, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, + 0x1f, 0x0a, 0x0b, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x5f, 0x73, 0x74, 0x6f, 0x70, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x53, 0x74, 0x6f, 0x70, + 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x22, 0x74, 0x0a, 0x12, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1b, 0x0a, 0x09, + 0x74, 0x63, 0x70, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x08, 0x74, 0x63, 0x70, 0x50, 0x6f, 0x72, 0x74, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x68, 0x74, 0x74, + 0x70, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x68, 0x74, + 0x74, 0x70, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x8f, 0x01, 0x0a, + 0x0a, 0x4c, 0x6f, 0x67, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, + 0x32, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x67, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x1a, 0x39, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xaf, + 0x03, 0x0a, 0x11, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x64, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x70, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, + 0x65, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x69, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x64, 0x69, 0x72, 0x12, 0x20, 0x0a, 0x03, 0x6c, 0x6f, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x67, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x52, 0x03, 0x6c, 0x6f, 0x67, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, + 0x68, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, + 0x12, 0x38, 0x0a, 0x0b, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x70, 0x62, 0x2e, 0x48, 0x65, 0x61, 0x6c, 0x74, + 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x68, + 0x65, 0x61, 0x6c, 0x74, 0x68, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x12, 0x23, 0x0a, 0x04, 0x68, 0x6f, + 0x6f, 0x6b, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x48, 0x6f, + 0x6f, 0x6b, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x04, 0x68, 0x6f, 0x6f, 0x6b, 0x12, + 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x3c, 0x0a, 0x07, 0x73, 0x79, 0x73, 0x63, 0x74, 0x6c, + 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x53, + 0x79, 0x73, 0x63, 0x74, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x73, 0x79, 0x73, + 0x63, 0x74, 0x6c, 0x73, 0x1a, 0x3a, 0x0a, 0x0c, 0x53, 0x79, 0x73, 0x63, 0x74, 0x6c, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0xe6, 0x02, 0x0a, 0x0f, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x63, 0x70, 0x75, 0x5f, 0x71, 0x75, 0x6f, 0x74, + 0x61, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0d, 0x63, + 0x70, 0x75, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x2a, 0x0a, 0x11, + 0x63, 0x70, 0x75, 0x5f, 0x71, 0x75, 0x6f, 0x74, 0x61, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0f, 0x63, 0x70, 0x75, 0x51, 0x75, 0x6f, 0x74, + 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x70, 0x75, 0x5f, + 0x62, 0x69, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x63, 0x70, 0x75, 0x42, + 0x69, 0x6e, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x6c, 0x69, + 0x6d, 0x69, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x6d, 0x65, 0x6d, 0x6f, 0x72, + 0x79, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, + 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, + 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, + 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x69, 0x6d, + 0x69, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, + 0x72, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x76, + 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x08, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0c, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, + 0x12, 0x27, 0x0a, 0x0f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x5f, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x18, 0x09, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xeb, 0x05, 0x0a, 0x08, 0x52, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x63, 0x70, 0x75, 0x5f, 0x71, 0x75, + 0x6f, 0x74, 0x61, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x52, + 0x0d, 0x63, 0x70, 0x75, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x2a, + 0x0a, 0x11, 0x63, 0x70, 0x75, 0x5f, 0x71, 0x75, 0x6f, 0x74, 0x61, 0x5f, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0f, 0x63, 0x70, 0x75, 0x51, 0x75, + 0x6f, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x27, 0x0a, 0x03, 0x63, 0x70, + 0x75, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x43, 0x70, 0x75, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x03, + 0x63, 0x70, 0x75, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x6c, 0x69, + 0x6d, 0x69, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x6d, 0x65, 0x6d, 0x6f, 0x72, + 0x79, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, + 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, + 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, + 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x69, 0x6d, + 0x69, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, + 0x72, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x76, + 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x08, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0c, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, + 0x12, 0x27, 0x0a, 0x0f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x73, 0x5f, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x18, 0x09, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x76, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x11, 0x76, 0x6f, 0x6c, + 0x75, 0x6d, 0x65, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x0a, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x4c, 0x69, 0x6d, + 0x69, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x50, + 0x6c, 0x61, 0x6e, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x53, 0x0a, 0x13, 0x76, 0x6f, 0x6c, 0x75, + 0x6d, 0x65, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, + 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x76, 0x6f, 0x6c, 0x75, + 0x6d, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x0a, + 0x08, 0x43, 0x70, 0x75, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x4e, 0x0a, 0x14, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x50, + 0x6c, 0x61, 0x6e, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x20, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, + 0x2e, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x50, 0x0a, 0x16, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x50, + 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x20, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x73, 0x0a, 0x06, 0x56, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x12, 0x2e, 0x0a, 0x06, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x16, 0x2e, 0x70, 0x62, 0x2e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x2e, 0x56, 0x6f, + 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x76, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x1a, 0x39, 0x0a, 0x0b, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x99, 0x09, 0x0a, + 0x0d, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, + 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x12, 0x35, 0x0a, 0x0a, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0a, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x6f, 0x64, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x6f, 0x64, 0x6e, + 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x78, 0x74, 0x72, 0x61, + 0x5f, 0x61, 0x72, 0x67, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x65, 0x78, 0x74, + 0x72, 0x61, 0x41, 0x72, 0x67, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x10, 0x0a, 0x03, + 0x65, 0x6e, 0x76, 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x65, 0x6e, 0x76, 0x12, 0x10, + 0x0a, 0x03, 0x64, 0x6e, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x64, 0x6e, 0x73, + 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x73, 0x18, + 0x0a, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x65, 0x78, 0x74, 0x72, 0x61, 0x48, 0x6f, 0x73, 0x74, + 0x73, 0x12, 0x3b, 0x0a, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x18, 0x0b, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x20, + 0x0a, 0x0b, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0c, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x6d, 0x6f, 0x64, 0x65, + 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x75, 0x73, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x65, 0x62, 0x75, 0x67, 0x18, 0x0e, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x05, 0x64, 0x65, 0x62, 0x75, 0x67, 0x12, 0x1c, 0x0a, 0x09, 0x6f, 0x70, + 0x65, 0x6e, 0x53, 0x74, 0x64, 0x69, 0x6e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x6f, + 0x70, 0x65, 0x6e, 0x53, 0x74, 0x64, 0x69, 0x6e, 0x12, 0x35, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, + 0x6c, 0x73, 0x18, 0x10, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, + 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4c, 0x61, 0x62, 0x65, + 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, + 0x41, 0x0a, 0x0a, 0x6e, 0x6f, 0x64, 0x65, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x11, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x6c, 0x61, 0x62, 0x65, 0x6c, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x6e, 0x6f, 0x64, 0x65, 0x6c, 0x61, 0x62, 0x65, + 0x6c, 0x73, 0x12, 0x43, 0x0a, 0x0f, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x5f, 0x73, 0x74, 0x72, + 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x62, + 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x53, + 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x0e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x53, + 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, + 0x13, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, + 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x6f, 0x64, 0x65, + 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x14, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x6e, + 0x6f, 0x64, 0x65, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x67, 0x6e, + 0x6f, 0x72, 0x65, 0x5f, 0x68, 0x6f, 0x6f, 0x6b, 0x18, 0x15, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, + 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x66, + 0x74, 0x65, 0x72, 0x5f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x16, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x0b, 0x61, 0x66, 0x74, 0x65, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x19, 0x0a, + 0x08, 0x72, 0x61, 0x77, 0x5f, 0x61, 0x72, 0x67, 0x73, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x07, 0x72, 0x61, 0x77, 0x41, 0x72, 0x67, 0x73, 0x12, 0x38, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x73, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, + 0x74, 0x73, 0x1a, 0x3b, 0x0a, 0x0d, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, + 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3d, 0x0a, 0x0f, 0x4e, 0x6f, + 0x64, 0x65, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x37, 0x0a, 0x09, 0x44, 0x61, 0x74, + 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x34, 0x0a, 0x08, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x08, + 0x0a, 0x04, 0x41, 0x55, 0x54, 0x4f, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x49, 0x4c, 0x4c, + 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x45, 0x41, 0x43, 0x48, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, + 0x47, 0x4c, 0x4f, 0x42, 0x41, 0x4c, 0x10, 0x03, 0x22, 0xf2, 0x02, 0x0a, 0x0e, 0x52, 0x65, 0x70, + 0x6c, 0x61, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2f, 0x0a, 0x09, 0x64, + 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, + 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x52, 0x09, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x12, 0x26, 0x0a, 0x0e, + 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x69, 0x6e, 0x68, 0x65, 0x72, 0x69, 0x74, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x69, 0x6e, 0x68, + 0x65, 0x72, 0x69, 0x74, 0x12, 0x49, 0x0a, 0x0d, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x5f, 0x6c, + 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x62, + 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, + 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x0c, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, + 0x30, 0x0a, 0x04, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x70, 0x62, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x63, 0x6f, 0x70, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, + 0x69, 0x64, 0x73, 0x1a, 0x3f, 0x0a, 0x11, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x4c, 0x61, 0x62, + 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x37, 0x0a, 0x09, 0x43, 0x6f, 0x70, 0x79, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x77, 0x0a, + 0x11, 0x43, 0x61, 0x63, 0x68, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, + 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x69, 0x6d, + 0x61, 0x67, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x69, 0x6d, 0x61, 0x67, + 0x65, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x74, 0x65, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x04, 0x73, 0x74, 0x65, 0x70, 0x22, 0x8e, 0x01, 0x0a, 0x12, 0x52, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x18, 0x0a, + 0x07, 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x70, 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x6f, 0x64, 0x65, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x73, 0x12, 0x12, 0x0a, + 0x04, 0x73, 0x74, 0x65, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x73, 0x74, 0x65, + 0x70, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x72, 0x75, 0x6e, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x05, 0x70, 0x72, 0x75, 0x6e, 0x65, 0x22, 0x21, 0x0a, 0x09, 0x43, 0x6f, 0x70, 0x79, 0x50, + 0x61, 0x74, 0x68, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x22, 0x90, 0x01, 0x0a, 0x0b, 0x43, + 0x6f, 0x70, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x36, 0x0a, 0x07, 0x74, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x62, + 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x54, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x74, 0x61, 0x72, 0x67, 0x65, + 0x74, 0x73, 0x1a, 0x49, 0x0a, 0x0c, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x23, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x50, 0x61, 0x74, + 0x68, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x87, 0x01, + 0x0a, 0x0b, 0x53, 0x65, 0x6e, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x10, 0x0a, + 0x03, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x73, 0x12, + 0x2d, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x70, 0x62, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x44, + 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x37, + 0x0a, 0x09, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x0b, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x22, 0xb9, 0x01, 0x0a, 0x11, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x6d, + 0x61, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x14, + 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x32, 0x0a, 0x0c, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x52, 0x0b, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x22, 0xdf, 0x02, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, + 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x70, + 0x6f, 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x6f, + 0x64, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, + 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x73, + 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, + 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x41, 0x0a, 0x07, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, + 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x07, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x6f, 0x6b, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x68, 0x6f, 0x6f, 0x6b, 0x12, 0x28, 0x0a, 0x08, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, + 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x08, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x1a, 0x3a, 0x0a, 0x0c, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, + 0x68, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x97, 0x01, 0x0a, 0x17, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x43, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x32, + 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, + 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x06, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x12, 0x32, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x43, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x06, + 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x79, 0x0a, 0x11, + 0x43, 0x61, 0x63, 0x68, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x60, 0x0a, 0x12, 0x52, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, 0x6d, + 0x61, 0x67, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1a, 0x0a, + 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x22, 0x56, 0x0a, 0x16, 0x52, 0x65, 0x6d, + 0x6f, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x02, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x12, 0x0a, + 0x04, 0x68, 0x6f, 0x6f, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x6f, + 0x6b, 0x22, 0x42, 0x0a, 0x1a, 0x44, 0x69, 0x73, 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x43, + 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, + 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, + 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2e, 0x0a, 0x16, 0x52, 0x65, 0x61, 0x6c, 0x6c, 0x6f, 0x63, + 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x87, 0x01, 0x0a, 0x0b, 0x43, 0x6f, 0x70, 0x79, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, + 0x47, 0x0a, 0x0b, 0x53, 0x65, 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x0e, + 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, + 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, + 0x74, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x4f, 0x0a, 0x16, 0x41, 0x74, 0x74, 0x61, + 0x63, 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, + 0x6e, 0x65, 0x72, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x9a, 0x01, 0x0a, 0x11, 0x52, 0x75, + 0x6e, 0x41, 0x6e, 0x64, 0x57, 0x61, 0x69, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x38, 0x0a, 0x0e, 0x64, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, + 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0d, 0x64, 0x65, 0x70, 0x6c, + 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x63, 0x6d, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x63, 0x6d, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x61, + 0x73, 0x79, 0x6e, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x61, 0x73, 0x79, 0x6e, + 0x63, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x54, + 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, 0x55, 0x0a, 0x17, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, + 0x6c, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, + 0x69, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x22, 0x53, 0x0a, + 0x17, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, + 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, + 0x0a, 0x04, 0x68, 0x6f, 0x6f, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x68, 0x6f, + 0x6f, 0x6b, 0x22, 0x62, 0x0a, 0x10, 0x4c, 0x6f, 0x67, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x69, 0x6c, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x69, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x69, + 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x69, 0x6e, 0x63, 0x65, + 0x12, 0x14, 0x0a, 0x05, 0x75, 0x6e, 0x74, 0x69, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x75, 0x6e, 0x74, 0x69, 0x6c, 0x22, 0x4c, 0x0a, 0x10, 0x4c, 0x6f, 0x67, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, + 0x64, 0x61, 0x74, 0x61, 0x22, 0xc0, 0x01, 0x0a, 0x17, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, + 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x12, + 0x12, 0x0a, 0x04, 0x65, 0x6e, 0x76, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x65, + 0x6e, 0x76, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x77, 0x6f, 0x72, 0x6b, 0x64, 0x69, 0x72, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x77, 0x6f, 0x72, 0x6b, 0x64, 0x69, 0x72, 0x12, 0x1d, 0x0a, + 0x0a, 0x6f, 0x70, 0x65, 0x6e, 0x5f, 0x73, 0x74, 0x64, 0x69, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x6e, 0x53, 0x74, 0x64, 0x69, 0x6e, 0x12, 0x19, 0x0a, 0x08, + 0x72, 0x65, 0x70, 0x6c, 0x5f, 0x63, 0x6d, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, + 0x72, 0x65, 0x70, 0x6c, 0x43, 0x6d, 0x64, 0x2a, 0x27, 0x0a, 0x06, 0x54, 0x72, 0x69, 0x4f, 0x70, + 0x74, 0x12, 0x08, 0x0a, 0x04, 0x4b, 0x45, 0x45, 0x50, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x54, + 0x52, 0x55, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x41, 0x4c, 0x53, 0x45, 0x10, 0x02, + 0x32, 0xb6, 0x11, 0x0a, 0x07, 0x43, 0x6f, 0x72, 0x65, 0x52, 0x50, 0x43, 0x12, 0x21, 0x0a, 0x04, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, + 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x00, 0x12, + 0x36, 0x0a, 0x12, 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x1a, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x00, 0x30, 0x01, 0x12, 0x36, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, 0x4e, + 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x16, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, + 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, + 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x22, 0x00, 0x12, + 0x3a, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, + 0x6b, 0x12, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x4e, 0x65, + 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x0b, 0x2e, 0x70, + 0x62, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x22, 0x00, 0x12, 0x3e, 0x0a, 0x11, 0x44, + 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, + 0x12, 0x1c, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x09, + 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x12, 0x26, 0x0a, 0x06, 0x41, + 0x64, 0x64, 0x50, 0x6f, 0x64, 0x12, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x64, 0x64, 0x50, 0x6f, + 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x07, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, + 0x64, 0x22, 0x00, 0x12, 0x2e, 0x0a, 0x09, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x50, 0x6f, 0x64, + 0x12, 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x50, 0x6f, 0x64, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, 0x74, + 0x79, 0x22, 0x00, 0x12, 0x26, 0x0a, 0x06, 0x47, 0x65, 0x74, 0x50, 0x6f, 0x64, 0x12, 0x11, 0x2e, + 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x1a, 0x07, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x64, 0x22, 0x00, 0x12, 0x21, 0x0a, 0x08, 0x4c, + 0x69, 0x73, 0x74, 0x50, 0x6f, 0x64, 0x73, 0x12, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, + 0x74, 0x79, 0x1a, 0x08, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x64, 0x73, 0x22, 0x00, 0x12, 0x36, + 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x50, 0x6f, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x12, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x1a, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x6f, 0x64, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x22, 0x00, 0x12, 0x29, 0x0a, 0x07, 0x41, 0x64, 0x64, 0x4e, 0x6f, 0x64, + 0x65, 0x12, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x64, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x08, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x22, + 0x00, 0x12, 0x30, 0x0a, 0x0a, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x12, + 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, 0x74, + 0x79, 0x22, 0x00, 0x12, 0x31, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x64, 0x4e, 0x6f, + 0x64, 0x65, 0x73, 0x12, 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x6f, 0x64, + 0x65, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x4e, + 0x6f, 0x64, 0x65, 0x73, 0x22, 0x00, 0x12, 0x29, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, + 0x65, 0x12, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x08, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x22, + 0x00, 0x12, 0x29, 0x0a, 0x07, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x2e, 0x70, + 0x62, 0x2e, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x1a, 0x08, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x22, 0x00, 0x12, 0x41, 0x0a, 0x0f, + 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, + 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x10, 0x2e, 0x70, 0x62, + 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, 0x00, 0x12, + 0x30, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x12, + 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x49, 0x44, + 0x1a, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x22, + 0x00, 0x12, 0x33, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, + 0x72, 0x73, 0x12, 0x10, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, + 0x72, 0x49, 0x44, 0x73, 0x1a, 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, + 0x6e, 0x65, 0x72, 0x73, 0x22, 0x00, 0x12, 0x3e, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, 0x12, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x1a, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, + 0x65, 0x72, 0x22, 0x00, 0x30, 0x01, 0x12, 0x3a, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x6f, + 0x64, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, 0x12, 0x12, 0x2e, 0x70, + 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x1a, 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, + 0x22, 0x00, 0x12, 0x3f, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, + 0x65, 0x72, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x10, 0x2e, 0x70, 0x62, 0x2e, 0x43, + 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x49, 0x44, 0x73, 0x1a, 0x14, 0x2e, 0x70, 0x62, + 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x00, 0x12, 0x4d, 0x0a, 0x13, 0x53, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, + 0x6e, 0x65, 0x72, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1e, 0x2e, 0x70, 0x62, 0x2e, + 0x53, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x14, 0x2e, 0x70, 0x62, 0x2e, + 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x15, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x20, 0x2e, 0x70, 0x62, + 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x20, 0x2e, + 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, + 0x00, 0x30, 0x01, 0x12, 0x2c, 0x0a, 0x04, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x0f, 0x2e, 0x70, 0x62, + 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x0f, 0x2e, 0x70, + 0x62, 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, + 0x01, 0x12, 0x2c, 0x0a, 0x04, 0x53, 0x65, 0x6e, 0x64, 0x12, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x53, + 0x65, 0x6e, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x0f, 0x2e, 0x70, 0x62, 0x2e, + 0x53, 0x65, 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, + 0x3e, 0x0a, 0x0a, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x15, 0x2e, + 0x70, 0x62, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, + 0x6d, 0x61, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, + 0x3e, 0x0a, 0x0a, 0x43, 0x61, 0x63, 0x68, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x15, 0x2e, + 0x70, 0x62, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x49, + 0x6d, 0x61, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, + 0x41, 0x0a, 0x0b, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x16, + 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x16, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, + 0x76, 0x65, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, + 0x30, 0x01, 0x12, 0x44, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x74, + 0x61, 0x69, 0x6e, 0x65, 0x72, 0x12, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, + 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x47, 0x0a, 0x10, 0x52, 0x65, 0x70, 0x6c, + 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x12, 0x12, 0x2e, 0x70, + 0x62, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x1a, 0x1b, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6e, + 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, + 0x01, 0x12, 0x4d, 0x0a, 0x0f, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, + 0x69, 0x6e, 0x65, 0x72, 0x12, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, + 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x1a, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x74, + 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, + 0x12, 0x59, 0x0a, 0x13, 0x44, 0x69, 0x73, 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x43, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x12, 0x1e, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x69, 0x73, + 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1e, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x69, 0x73, + 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x50, 0x0a, 0x10, 0x43, + 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x12, + 0x1b, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x43, 0x6f, 0x6e, 0x74, + 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1b, 0x2e, 0x70, + 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, + 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x51, 0x0a, + 0x10, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, + 0x72, 0x12, 0x1b, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x43, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1a, + 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x74, 0x74, 0x61, 0x63, 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, + 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, + 0x12, 0x43, 0x0a, 0x0f, 0x52, 0x65, 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x12, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x61, 0x6c, 0x6c, 0x6f, 0x63, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x61, + 0x6c, 0x6c, 0x6f, 0x63, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x00, 0x12, 0x3b, 0x0a, 0x09, 0x4c, 0x6f, 0x67, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x12, 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x67, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x6f, + 0x67, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, + 0x30, 0x01, 0x12, 0x45, 0x0a, 0x0a, 0x52, 0x75, 0x6e, 0x41, 0x6e, 0x64, 0x57, 0x61, 0x69, 0x74, + 0x12, 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x75, 0x6e, 0x41, 0x6e, 0x64, 0x57, 0x61, 0x69, 0x74, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x74, 0x74, + 0x61, 0x63, 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, +} + +var ( + file_core_proto_rawDescOnce sync.Once + file_core_proto_rawDescData = file_core_proto_rawDesc +) + +func file_core_proto_rawDescGZIP() []byte { + file_core_proto_rawDescOnce.Do(func() { + file_core_proto_rawDescData = protoimpl.X.CompressGZIP(file_core_proto_rawDescData) + }) + return file_core_proto_rawDescData +} + +var file_core_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_core_proto_msgTypes = make([]protoimpl.MessageInfo, 122) +var file_core_proto_goTypes = []interface{}{ + (TriOpt)(0), // 0: pb.TriOpt + (BuildImageOptions_BuildMethod)(0), // 1: pb.BuildImageOptions.BuildMethod + (DeployOptions_Strategy)(0), // 2: pb.DeployOptions.Strategy + (*Empty)(nil), // 3: pb.Empty + (*CoreInfo)(nil), // 4: pb.CoreInfo + (*ServiceStatus)(nil), // 5: pb.ServiceStatus + (*ListContainersOptions)(nil), // 6: pb.ListContainersOptions + (*Pod)(nil), // 7: pb.Pod + (*Pods)(nil), // 8: pb.Pods + (*PodResource)(nil), // 9: pb.PodResource + (*NodeResource)(nil), // 10: pb.NodeResource + (*ListNetworkOptions)(nil), // 11: pb.ListNetworkOptions + (*ConnectNetworkOptions)(nil), // 12: pb.ConnectNetworkOptions + (*DisconnectNetworkOptions)(nil), // 13: pb.DisconnectNetworkOptions + (*Network)(nil), // 14: pb.Network + (*Networks)(nil), // 15: pb.Networks + (*Node)(nil), // 16: pb.Node + (*Nodes)(nil), // 17: pb.Nodes + (*NodeAvailable)(nil), // 18: pb.NodeAvailable + (*SetNodeOptions)(nil), // 19: pb.SetNodeOptions + (*Container)(nil), // 20: pb.Container + (*ContainerStatus)(nil), // 21: pb.ContainerStatus + (*ContainersStatus)(nil), // 22: pb.ContainersStatus + (*ContainerStatusStreamMessage)(nil), // 23: pb.ContainerStatusStreamMessage + (*SetContainersStatusOptions)(nil), // 24: pb.SetContainersStatusOptions + (*ContainerStatusStreamOptions)(nil), // 25: pb.ContainerStatusStreamOptions + (*Containers)(nil), // 26: pb.Containers + (*ContainerID)(nil), // 27: pb.ContainerID + (*ContainerIDs)(nil), // 28: pb.ContainerIDs + (*RemoveContainerOptions)(nil), // 29: pb.RemoveContainerOptions + (*DissociateContainerOptions)(nil), // 30: pb.DissociateContainerOptions + (*ReallocOptions)(nil), // 31: pb.ReallocOptions + (*AddPodOptions)(nil), // 32: pb.AddPodOptions + (*RemovePodOptions)(nil), // 33: pb.RemovePodOptions + (*GetPodOptions)(nil), // 34: pb.GetPodOptions + (*AddNodeOptions)(nil), // 35: pb.AddNodeOptions + (*RemoveNodeOptions)(nil), // 36: pb.RemoveNodeOptions + (*GetNodeOptions)(nil), // 37: pb.GetNodeOptions + (*GetNodeResourceOptions)(nil), // 38: pb.GetNodeResourceOptions + (*ListNodesOptions)(nil), // 39: pb.ListNodesOptions + (*Build)(nil), // 40: pb.Build + (*Builds)(nil), // 41: pb.Builds + (*BuildImageOptions)(nil), // 42: pb.BuildImageOptions + (*HookOptions)(nil), // 43: pb.HookOptions + (*HealthCheckOptions)(nil), // 44: pb.HealthCheckOptions + (*LogOptions)(nil), // 45: pb.LogOptions + (*EntrypointOptions)(nil), // 46: pb.EntrypointOptions + (*ResourceOptions)(nil), // 47: pb.ResourceOptions + (*Resource)(nil), // 48: pb.Resource + (*Volume)(nil), // 49: pb.Volume + (*DeployOptions)(nil), // 50: pb.DeployOptions + (*ReplaceOptions)(nil), // 51: pb.ReplaceOptions + (*CacheImageOptions)(nil), // 52: pb.CacheImageOptions + (*RemoveImageOptions)(nil), // 53: pb.RemoveImageOptions + (*CopyPaths)(nil), // 54: pb.CopyPaths + (*CopyOptions)(nil), // 55: pb.CopyOptions + (*SendOptions)(nil), // 56: pb.SendOptions + (*ErrorDetail)(nil), // 57: pb.ErrorDetail + (*BuildImageMessage)(nil), // 58: pb.BuildImageMessage + (*CreateContainerMessage)(nil), // 59: pb.CreateContainerMessage + (*ReplaceContainerMessage)(nil), // 60: pb.ReplaceContainerMessage + (*CacheImageMessage)(nil), // 61: pb.CacheImageMessage + (*RemoveImageMessage)(nil), // 62: pb.RemoveImageMessage + (*RemoveContainerMessage)(nil), // 63: pb.RemoveContainerMessage + (*DissociateContainerMessage)(nil), // 64: pb.DissociateContainerMessage + (*ReallocResourceMessage)(nil), // 65: pb.ReallocResourceMessage + (*CopyMessage)(nil), // 66: pb.CopyMessage + (*SendMessage)(nil), // 67: pb.SendMessage + (*AttachContainerMessage)(nil), // 68: pb.AttachContainerMessage + (*RunAndWaitOptions)(nil), // 69: pb.RunAndWaitOptions + (*ControlContainerOptions)(nil), // 70: pb.ControlContainerOptions + (*ControlContainerMessage)(nil), // 71: pb.ControlContainerMessage + (*LogStreamOptions)(nil), // 72: pb.LogStreamOptions + (*LogStreamMessage)(nil), // 73: pb.LogStreamMessage + (*ExecuteContainerOptions)(nil), // 74: pb.ExecuteContainerOptions + nil, // 75: pb.ListContainersOptions.LabelsEntry + nil, // 76: pb.PodResource.CpuPercentsEntry + nil, // 77: pb.PodResource.MemoryPercentsEntry + nil, // 78: pb.PodResource.VerificationsEntry + nil, // 79: pb.PodResource.DetailsEntry + nil, // 80: pb.PodResource.StoragePercentsEntry + nil, // 81: pb.PodResource.VolumePercentsEntry + nil, // 82: pb.Node.CpuEntry + nil, // 83: pb.Node.LabelsEntry + nil, // 84: pb.Node.InitCpuEntry + nil, // 85: pb.Node.NumaEntry + nil, // 86: pb.Node.NumaMemoryEntry + nil, // 87: pb.Node.InitVolumeEntry + nil, // 88: pb.Node.VolumeEntry + nil, // 89: pb.SetNodeOptions.DeltaCpuEntry + nil, // 90: pb.SetNodeOptions.DeltaNumaMemoryEntry + nil, // 91: pb.SetNodeOptions.NumaEntry + nil, // 92: pb.SetNodeOptions.LabelsEntry + nil, // 93: pb.SetNodeOptions.DeltaVolumeEntry + nil, // 94: pb.Container.LabelsEntry + nil, // 95: pb.Container.PublishEntry + nil, // 96: pb.ContainerStatus.NetworksEntry + nil, // 97: pb.ContainerStatusStreamOptions.LabelsEntry + nil, // 98: pb.AddNodeOptions.LabelsEntry + nil, // 99: pb.AddNodeOptions.NumaEntry + nil, // 100: pb.AddNodeOptions.NumaMemoryEntry + nil, // 101: pb.AddNodeOptions.VolumeMapEntry + nil, // 102: pb.GetNodeOptions.LabelsEntry + nil, // 103: pb.ListNodesOptions.LabelsEntry + nil, // 104: pb.Build.EnvsEntry + nil, // 105: pb.Build.ArgsEntry + nil, // 106: pb.Build.LabelsEntry + nil, // 107: pb.Build.ArtifactsEntry + nil, // 108: pb.Build.CacheEntry + nil, // 109: pb.Builds.BuildsEntry + nil, // 110: pb.LogOptions.ConfigEntry + nil, // 111: pb.EntrypointOptions.SysctlsEntry + nil, // 112: pb.Resource.CpuEntry + nil, // 113: pb.Resource.VolumePlanLimitEntry + nil, // 114: pb.Resource.VolumePlanRequestEntry + nil, // 115: pb.Volume.VolumeEntry + nil, // 116: pb.DeployOptions.NetworksEntry + nil, // 117: pb.DeployOptions.LabelsEntry + nil, // 118: pb.DeployOptions.NodelabelsEntry + nil, // 119: pb.DeployOptions.DataEntry + nil, // 120: pb.ReplaceOptions.FilterLabelsEntry + nil, // 121: pb.ReplaceOptions.CopyEntry + nil, // 122: pb.CopyOptions.TargetsEntry + nil, // 123: pb.SendOptions.DataEntry + nil, // 124: pb.CreateContainerMessage.PublishEntry +} +var file_core_proto_depIdxs = []int32{ + 75, // 0: pb.ListContainersOptions.labels:type_name -> pb.ListContainersOptions.LabelsEntry + 7, // 1: pb.Pods.pods:type_name -> pb.Pod + 76, // 2: pb.PodResource.cpu_percents:type_name -> pb.PodResource.CpuPercentsEntry + 77, // 3: pb.PodResource.memory_percents:type_name -> pb.PodResource.MemoryPercentsEntry + 78, // 4: pb.PodResource.verifications:type_name -> pb.PodResource.VerificationsEntry + 79, // 5: pb.PodResource.details:type_name -> pb.PodResource.DetailsEntry + 80, // 6: pb.PodResource.storage_percents:type_name -> pb.PodResource.StoragePercentsEntry + 81, // 7: pb.PodResource.volume_percents:type_name -> pb.PodResource.VolumePercentsEntry + 14, // 8: pb.Networks.networks:type_name -> pb.Network + 82, // 9: pb.Node.cpu:type_name -> pb.Node.CpuEntry + 83, // 10: pb.Node.labels:type_name -> pb.Node.LabelsEntry + 84, // 11: pb.Node.init_cpu:type_name -> pb.Node.InitCpuEntry + 85, // 12: pb.Node.numa:type_name -> pb.Node.NumaEntry + 86, // 13: pb.Node.numa_memory:type_name -> pb.Node.NumaMemoryEntry + 87, // 14: pb.Node.init_volume:type_name -> pb.Node.InitVolumeEntry + 88, // 15: pb.Node.volume:type_name -> pb.Node.VolumeEntry + 16, // 16: pb.Nodes.nodes:type_name -> pb.Node + 0, // 17: pb.SetNodeOptions.status_opt:type_name -> pb.TriOpt + 89, // 18: pb.SetNodeOptions.delta_cpu:type_name -> pb.SetNodeOptions.DeltaCpuEntry + 90, // 19: pb.SetNodeOptions.delta_numa_memory:type_name -> pb.SetNodeOptions.DeltaNumaMemoryEntry + 91, // 20: pb.SetNodeOptions.numa:type_name -> pb.SetNodeOptions.NumaEntry + 92, // 21: pb.SetNodeOptions.labels:type_name -> pb.SetNodeOptions.LabelsEntry + 93, // 22: pb.SetNodeOptions.delta_volume:type_name -> pb.SetNodeOptions.DeltaVolumeEntry + 94, // 23: pb.Container.labels:type_name -> pb.Container.LabelsEntry + 95, // 24: pb.Container.publish:type_name -> pb.Container.PublishEntry + 21, // 25: pb.Container.status:type_name -> pb.ContainerStatus + 48, // 26: pb.Container.resource:type_name -> pb.Resource + 96, // 27: pb.ContainerStatus.networks:type_name -> pb.ContainerStatus.NetworksEntry + 21, // 28: pb.ContainersStatus.status:type_name -> pb.ContainerStatus + 20, // 29: pb.ContainerStatusStreamMessage.container:type_name -> pb.Container + 21, // 30: pb.ContainerStatusStreamMessage.status:type_name -> pb.ContainerStatus + 21, // 31: pb.SetContainersStatusOptions.status:type_name -> pb.ContainerStatus + 97, // 32: pb.ContainerStatusStreamOptions.labels:type_name -> pb.ContainerStatusStreamOptions.LabelsEntry + 20, // 33: pb.Containers.containers:type_name -> pb.Container + 0, // 34: pb.ReallocOptions.bind_cpu_opt:type_name -> pb.TriOpt + 47, // 35: pb.ReallocOptions.resource_opts:type_name -> pb.ResourceOptions + 98, // 36: pb.AddNodeOptions.labels:type_name -> pb.AddNodeOptions.LabelsEntry + 99, // 37: pb.AddNodeOptions.numa:type_name -> pb.AddNodeOptions.NumaEntry + 100, // 38: pb.AddNodeOptions.numa_memory:type_name -> pb.AddNodeOptions.NumaMemoryEntry + 101, // 39: pb.AddNodeOptions.volume_map:type_name -> pb.AddNodeOptions.VolumeMapEntry + 102, // 40: pb.GetNodeOptions.labels:type_name -> pb.GetNodeOptions.LabelsEntry + 37, // 41: pb.GetNodeResourceOptions.opts:type_name -> pb.GetNodeOptions + 103, // 42: pb.ListNodesOptions.labels:type_name -> pb.ListNodesOptions.LabelsEntry + 104, // 43: pb.Build.envs:type_name -> pb.Build.EnvsEntry + 105, // 44: pb.Build.args:type_name -> pb.Build.ArgsEntry + 106, // 45: pb.Build.labels:type_name -> pb.Build.LabelsEntry + 107, // 46: pb.Build.artifacts:type_name -> pb.Build.ArtifactsEntry + 108, // 47: pb.Build.cache:type_name -> pb.Build.CacheEntry + 109, // 48: pb.Builds.builds:type_name -> pb.Builds.BuildsEntry + 41, // 49: pb.BuildImageOptions.builds:type_name -> pb.Builds + 1, // 50: pb.BuildImageOptions.build_method:type_name -> pb.BuildImageOptions.BuildMethod + 110, // 51: pb.LogOptions.config:type_name -> pb.LogOptions.ConfigEntry + 45, // 52: pb.EntrypointOptions.log:type_name -> pb.LogOptions + 44, // 53: pb.EntrypointOptions.healthcheck:type_name -> pb.HealthCheckOptions + 43, // 54: pb.EntrypointOptions.hook:type_name -> pb.HookOptions + 111, // 55: pb.EntrypointOptions.sysctls:type_name -> pb.EntrypointOptions.SysctlsEntry + 112, // 56: pb.Resource.cpu:type_name -> pb.Resource.CpuEntry + 113, // 57: pb.Resource.volume_plan_limit:type_name -> pb.Resource.VolumePlanLimitEntry + 114, // 58: pb.Resource.volume_plan_request:type_name -> pb.Resource.VolumePlanRequestEntry + 115, // 59: pb.Volume.volume:type_name -> pb.Volume.VolumeEntry + 46, // 60: pb.DeployOptions.entrypoint:type_name -> pb.EntrypointOptions + 116, // 61: pb.DeployOptions.networks:type_name -> pb.DeployOptions.NetworksEntry + 117, // 62: pb.DeployOptions.labels:type_name -> pb.DeployOptions.LabelsEntry + 118, // 63: pb.DeployOptions.nodelabels:type_name -> pb.DeployOptions.NodelabelsEntry + 2, // 64: pb.DeployOptions.deploy_strategy:type_name -> pb.DeployOptions.Strategy + 119, // 65: pb.DeployOptions.data:type_name -> pb.DeployOptions.DataEntry + 47, // 66: pb.DeployOptions.resource_opts:type_name -> pb.ResourceOptions + 50, // 67: pb.ReplaceOptions.deployOpt:type_name -> pb.DeployOptions + 120, // 68: pb.ReplaceOptions.filter_labels:type_name -> pb.ReplaceOptions.FilterLabelsEntry + 121, // 69: pb.ReplaceOptions.copy:type_name -> pb.ReplaceOptions.CopyEntry + 122, // 70: pb.CopyOptions.targets:type_name -> pb.CopyOptions.TargetsEntry + 123, // 71: pb.SendOptions.data:type_name -> pb.SendOptions.DataEntry + 57, // 72: pb.BuildImageMessage.error_detail:type_name -> pb.ErrorDetail + 124, // 73: pb.CreateContainerMessage.publish:type_name -> pb.CreateContainerMessage.PublishEntry + 48, // 74: pb.CreateContainerMessage.resource:type_name -> pb.Resource + 59, // 75: pb.ReplaceContainerMessage.create:type_name -> pb.CreateContainerMessage + 63, // 76: pb.ReplaceContainerMessage.remove:type_name -> pb.RemoveContainerMessage + 50, // 77: pb.RunAndWaitOptions.deploy_options:type_name -> pb.DeployOptions + 40, // 78: pb.Builds.BuildsEntry.value:type_name -> pb.Build + 49, // 79: pb.Resource.VolumePlanLimitEntry.value:type_name -> pb.Volume + 49, // 80: pb.Resource.VolumePlanRequestEntry.value:type_name -> pb.Volume + 54, // 81: pb.CopyOptions.TargetsEntry.value:type_name -> pb.CopyPaths + 3, // 82: pb.CoreRPC.Info:input_type -> pb.Empty + 3, // 83: pb.CoreRPC.WatchServiceStatus:input_type -> pb.Empty + 11, // 84: pb.CoreRPC.ListNetworks:input_type -> pb.ListNetworkOptions + 12, // 85: pb.CoreRPC.ConnectNetwork:input_type -> pb.ConnectNetworkOptions + 13, // 86: pb.CoreRPC.DisconnectNetwork:input_type -> pb.DisconnectNetworkOptions + 32, // 87: pb.CoreRPC.AddPod:input_type -> pb.AddPodOptions + 33, // 88: pb.CoreRPC.RemovePod:input_type -> pb.RemovePodOptions + 34, // 89: pb.CoreRPC.GetPod:input_type -> pb.GetPodOptions + 3, // 90: pb.CoreRPC.ListPods:input_type -> pb.Empty + 34, // 91: pb.CoreRPC.GetPodResource:input_type -> pb.GetPodOptions + 35, // 92: pb.CoreRPC.AddNode:input_type -> pb.AddNodeOptions + 36, // 93: pb.CoreRPC.RemoveNode:input_type -> pb.RemoveNodeOptions + 39, // 94: pb.CoreRPC.ListPodNodes:input_type -> pb.ListNodesOptions + 37, // 95: pb.CoreRPC.GetNode:input_type -> pb.GetNodeOptions + 19, // 96: pb.CoreRPC.SetNode:input_type -> pb.SetNodeOptions + 38, // 97: pb.CoreRPC.GetNodeResource:input_type -> pb.GetNodeResourceOptions + 27, // 98: pb.CoreRPC.GetContainer:input_type -> pb.ContainerID + 28, // 99: pb.CoreRPC.GetContainers:input_type -> pb.ContainerIDs + 6, // 100: pb.CoreRPC.ListContainers:input_type -> pb.ListContainersOptions + 37, // 101: pb.CoreRPC.ListNodeContainers:input_type -> pb.GetNodeOptions + 28, // 102: pb.CoreRPC.GetContainersStatus:input_type -> pb.ContainerIDs + 24, // 103: pb.CoreRPC.SetContainersStatus:input_type -> pb.SetContainersStatusOptions + 25, // 104: pb.CoreRPC.ContainerStatusStream:input_type -> pb.ContainerStatusStreamOptions + 55, // 105: pb.CoreRPC.Copy:input_type -> pb.CopyOptions + 56, // 106: pb.CoreRPC.Send:input_type -> pb.SendOptions + 42, // 107: pb.CoreRPC.BuildImage:input_type -> pb.BuildImageOptions + 52, // 108: pb.CoreRPC.CacheImage:input_type -> pb.CacheImageOptions + 53, // 109: pb.CoreRPC.RemoveImage:input_type -> pb.RemoveImageOptions + 50, // 110: pb.CoreRPC.CreateContainer:input_type -> pb.DeployOptions + 51, // 111: pb.CoreRPC.ReplaceContainer:input_type -> pb.ReplaceOptions + 29, // 112: pb.CoreRPC.RemoveContainer:input_type -> pb.RemoveContainerOptions + 30, // 113: pb.CoreRPC.DissociateContainer:input_type -> pb.DissociateContainerOptions + 70, // 114: pb.CoreRPC.ControlContainer:input_type -> pb.ControlContainerOptions + 74, // 115: pb.CoreRPC.ExecuteContainer:input_type -> pb.ExecuteContainerOptions + 31, // 116: pb.CoreRPC.ReallocResource:input_type -> pb.ReallocOptions + 72, // 117: pb.CoreRPC.LogStream:input_type -> pb.LogStreamOptions + 69, // 118: pb.CoreRPC.RunAndWait:input_type -> pb.RunAndWaitOptions + 4, // 119: pb.CoreRPC.Info:output_type -> pb.CoreInfo + 5, // 120: pb.CoreRPC.WatchServiceStatus:output_type -> pb.ServiceStatus + 15, // 121: pb.CoreRPC.ListNetworks:output_type -> pb.Networks + 14, // 122: pb.CoreRPC.ConnectNetwork:output_type -> pb.Network + 3, // 123: pb.CoreRPC.DisconnectNetwork:output_type -> pb.Empty + 7, // 124: pb.CoreRPC.AddPod:output_type -> pb.Pod + 3, // 125: pb.CoreRPC.RemovePod:output_type -> pb.Empty + 7, // 126: pb.CoreRPC.GetPod:output_type -> pb.Pod + 8, // 127: pb.CoreRPC.ListPods:output_type -> pb.Pods + 9, // 128: pb.CoreRPC.GetPodResource:output_type -> pb.PodResource + 16, // 129: pb.CoreRPC.AddNode:output_type -> pb.Node + 3, // 130: pb.CoreRPC.RemoveNode:output_type -> pb.Empty + 17, // 131: pb.CoreRPC.ListPodNodes:output_type -> pb.Nodes + 16, // 132: pb.CoreRPC.GetNode:output_type -> pb.Node + 16, // 133: pb.CoreRPC.SetNode:output_type -> pb.Node + 10, // 134: pb.CoreRPC.GetNodeResource:output_type -> pb.NodeResource + 20, // 135: pb.CoreRPC.GetContainer:output_type -> pb.Container + 26, // 136: pb.CoreRPC.GetContainers:output_type -> pb.Containers + 20, // 137: pb.CoreRPC.ListContainers:output_type -> pb.Container + 26, // 138: pb.CoreRPC.ListNodeContainers:output_type -> pb.Containers + 22, // 139: pb.CoreRPC.GetContainersStatus:output_type -> pb.ContainersStatus + 22, // 140: pb.CoreRPC.SetContainersStatus:output_type -> pb.ContainersStatus + 23, // 141: pb.CoreRPC.ContainerStatusStream:output_type -> pb.ContainerStatusStreamMessage + 66, // 142: pb.CoreRPC.Copy:output_type -> pb.CopyMessage + 67, // 143: pb.CoreRPC.Send:output_type -> pb.SendMessage + 58, // 144: pb.CoreRPC.BuildImage:output_type -> pb.BuildImageMessage + 61, // 145: pb.CoreRPC.CacheImage:output_type -> pb.CacheImageMessage + 62, // 146: pb.CoreRPC.RemoveImage:output_type -> pb.RemoveImageMessage + 59, // 147: pb.CoreRPC.CreateContainer:output_type -> pb.CreateContainerMessage + 60, // 148: pb.CoreRPC.ReplaceContainer:output_type -> pb.ReplaceContainerMessage + 63, // 149: pb.CoreRPC.RemoveContainer:output_type -> pb.RemoveContainerMessage + 64, // 150: pb.CoreRPC.DissociateContainer:output_type -> pb.DissociateContainerMessage + 71, // 151: pb.CoreRPC.ControlContainer:output_type -> pb.ControlContainerMessage + 68, // 152: pb.CoreRPC.ExecuteContainer:output_type -> pb.AttachContainerMessage + 65, // 153: pb.CoreRPC.ReallocResource:output_type -> pb.ReallocResourceMessage + 73, // 154: pb.CoreRPC.LogStream:output_type -> pb.LogStreamMessage + 68, // 155: pb.CoreRPC.RunAndWait:output_type -> pb.AttachContainerMessage + 119, // [119:156] is the sub-list for method output_type + 82, // [82:119] is the sub-list for method input_type + 82, // [82:82] is the sub-list for extension type_name + 82, // [82:82] is the sub-list for extension extendee + 0, // [0:82] is the sub-list for field type_name +} + +func init() { file_core_proto_init() } +func file_core_proto_init() { + if File_core_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_core_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Empty); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CoreInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ServiceStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListContainersOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Pod); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Pods); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PodResource); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*NodeResource); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListNetworkOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ConnectNetworkOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DisconnectNetworkOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Network); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Networks); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Node); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Nodes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*NodeAvailable); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SetNodeOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Container); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ContainerStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ContainersStatus); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ContainerStatusStreamMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SetContainersStatusOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ContainerStatusStreamOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Containers); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ContainerID); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ContainerIDs); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RemoveContainerOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DissociateContainerOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReallocOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AddPodOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RemovePodOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetPodOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AddNodeOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RemoveNodeOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetNodeOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetNodeResourceOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListNodesOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Build); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Builds); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BuildImageOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*HookOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*HealthCheckOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LogOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*EntrypointOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResourceOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Resource); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Volume); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeployOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReplaceOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CacheImageOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RemoveImageOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CopyPaths); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CopyOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SendOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ErrorDetail); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BuildImageMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateContainerMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReplaceContainerMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CacheImageMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RemoveImageMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RemoveContainerMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DissociateContainerMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReallocResourceMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CopyMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SendMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AttachContainerMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*RunAndWaitOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ControlContainerOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ControlContainerMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LogStreamOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LogStreamMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_core_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ExecuteContainerOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_core_proto_rawDesc, + NumEnums: 3, + NumMessages: 122, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_core_proto_goTypes, + DependencyIndexes: file_core_proto_depIdxs, + EnumInfos: file_core_proto_enumTypes, + MessageInfos: file_core_proto_msgTypes, + }.Build() + File_core_proto = out.File + file_core_proto_rawDesc = nil + file_core_proto_goTypes = nil + file_core_proto_depIdxs = nil } // Reference imports to suppress errors if they are not otherwise used. var _ context.Context -var _ grpc.ClientConn +var _ grpc.ClientConnInterface // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion4 +const _ = grpc.SupportPackageIsVersion6 // CoreRPCClient is the client API for CoreRPC service. // @@ -5400,16 +7774,16 @@ type CoreRPCClient interface { DissociateContainer(ctx context.Context, in *DissociateContainerOptions, opts ...grpc.CallOption) (CoreRPC_DissociateContainerClient, error) ControlContainer(ctx context.Context, in *ControlContainerOptions, opts ...grpc.CallOption) (CoreRPC_ControlContainerClient, error) ExecuteContainer(ctx context.Context, opts ...grpc.CallOption) (CoreRPC_ExecuteContainerClient, error) - ReallocResource(ctx context.Context, in *ReallocOptions, opts ...grpc.CallOption) (CoreRPC_ReallocResourceClient, error) + ReallocResource(ctx context.Context, in *ReallocOptions, opts ...grpc.CallOption) (*ReallocResourceMessage, error) LogStream(ctx context.Context, in *LogStreamOptions, opts ...grpc.CallOption) (CoreRPC_LogStreamClient, error) RunAndWait(ctx context.Context, opts ...grpc.CallOption) (CoreRPC_RunAndWaitClient, error) } type coreRPCClient struct { - cc *grpc.ClientConn + cc grpc.ClientConnInterface } -func NewCoreRPCClient(cc *grpc.ClientConn) CoreRPCClient { +func NewCoreRPCClient(cc grpc.ClientConnInterface) CoreRPCClient { return &coreRPCClient{cc} } @@ -6040,40 +8414,17 @@ func (x *coreRPCExecuteContainerClient) Recv() (*AttachContainerMessage, error) return m, nil } -func (c *coreRPCClient) ReallocResource(ctx context.Context, in *ReallocOptions, opts ...grpc.CallOption) (CoreRPC_ReallocResourceClient, error) { - stream, err := c.cc.NewStream(ctx, &_CoreRPC_serviceDesc.Streams[14], "/pb.CoreRPC/ReallocResource", opts...) +func (c *coreRPCClient) ReallocResource(ctx context.Context, in *ReallocOptions, opts ...grpc.CallOption) (*ReallocResourceMessage, error) { + out := new(ReallocResourceMessage) + err := c.cc.Invoke(ctx, "/pb.CoreRPC/ReallocResource", in, out, opts...) if err != nil { return nil, err } - x := &coreRPCReallocResourceClient{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 CoreRPC_ReallocResourceClient interface { - Recv() (*ReallocResourceMessage, error) - grpc.ClientStream -} - -type coreRPCReallocResourceClient struct { - grpc.ClientStream -} - -func (x *coreRPCReallocResourceClient) Recv() (*ReallocResourceMessage, error) { - m := new(ReallocResourceMessage) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil + return out, nil } func (c *coreRPCClient) LogStream(ctx context.Context, in *LogStreamOptions, opts ...grpc.CallOption) (CoreRPC_LogStreamClient, error) { - stream, err := c.cc.NewStream(ctx, &_CoreRPC_serviceDesc.Streams[15], "/pb.CoreRPC/LogStream", opts...) + stream, err := c.cc.NewStream(ctx, &_CoreRPC_serviceDesc.Streams[14], "/pb.CoreRPC/LogStream", opts...) if err != nil { return nil, err } @@ -6105,7 +8456,7 @@ func (x *coreRPCLogStreamClient) Recv() (*LogStreamMessage, error) { } func (c *coreRPCClient) RunAndWait(ctx context.Context, opts ...grpc.CallOption) (CoreRPC_RunAndWaitClient, error) { - stream, err := c.cc.NewStream(ctx, &_CoreRPC_serviceDesc.Streams[16], "/pb.CoreRPC/RunAndWait", opts...) + stream, err := c.cc.NewStream(ctx, &_CoreRPC_serviceDesc.Streams[15], "/pb.CoreRPC/RunAndWait", opts...) if err != nil { return nil, err } @@ -6171,7 +8522,7 @@ type CoreRPCServer interface { DissociateContainer(*DissociateContainerOptions, CoreRPC_DissociateContainerServer) error ControlContainer(*ControlContainerOptions, CoreRPC_ControlContainerServer) error ExecuteContainer(CoreRPC_ExecuteContainerServer) error - ReallocResource(*ReallocOptions, CoreRPC_ReallocResourceServer) error + ReallocResource(context.Context, *ReallocOptions) (*ReallocResourceMessage, error) LogStream(*LogStreamOptions, CoreRPC_LogStreamServer) error RunAndWait(CoreRPC_RunAndWaitServer) error } @@ -6180,115 +8531,115 @@ type CoreRPCServer interface { type UnimplementedCoreRPCServer struct { } -func (*UnimplementedCoreRPCServer) Info(ctx context.Context, req *Empty) (*CoreInfo, error) { +func (*UnimplementedCoreRPCServer) Info(context.Context, *Empty) (*CoreInfo, error) { return nil, status.Errorf(codes.Unimplemented, "method Info not implemented") } -func (*UnimplementedCoreRPCServer) WatchServiceStatus(req *Empty, srv CoreRPC_WatchServiceStatusServer) error { +func (*UnimplementedCoreRPCServer) WatchServiceStatus(*Empty, CoreRPC_WatchServiceStatusServer) error { return status.Errorf(codes.Unimplemented, "method WatchServiceStatus not implemented") } -func (*UnimplementedCoreRPCServer) ListNetworks(ctx context.Context, req *ListNetworkOptions) (*Networks, error) { +func (*UnimplementedCoreRPCServer) ListNetworks(context.Context, *ListNetworkOptions) (*Networks, error) { return nil, status.Errorf(codes.Unimplemented, "method ListNetworks not implemented") } -func (*UnimplementedCoreRPCServer) ConnectNetwork(ctx context.Context, req *ConnectNetworkOptions) (*Network, error) { +func (*UnimplementedCoreRPCServer) ConnectNetwork(context.Context, *ConnectNetworkOptions) (*Network, error) { return nil, status.Errorf(codes.Unimplemented, "method ConnectNetwork not implemented") } -func (*UnimplementedCoreRPCServer) DisconnectNetwork(ctx context.Context, req *DisconnectNetworkOptions) (*Empty, error) { +func (*UnimplementedCoreRPCServer) DisconnectNetwork(context.Context, *DisconnectNetworkOptions) (*Empty, error) { return nil, status.Errorf(codes.Unimplemented, "method DisconnectNetwork not implemented") } -func (*UnimplementedCoreRPCServer) AddPod(ctx context.Context, req *AddPodOptions) (*Pod, error) { +func (*UnimplementedCoreRPCServer) AddPod(context.Context, *AddPodOptions) (*Pod, error) { return nil, status.Errorf(codes.Unimplemented, "method AddPod not implemented") } -func (*UnimplementedCoreRPCServer) RemovePod(ctx context.Context, req *RemovePodOptions) (*Empty, error) { +func (*UnimplementedCoreRPCServer) RemovePod(context.Context, *RemovePodOptions) (*Empty, error) { return nil, status.Errorf(codes.Unimplemented, "method RemovePod not implemented") } -func (*UnimplementedCoreRPCServer) GetPod(ctx context.Context, req *GetPodOptions) (*Pod, error) { +func (*UnimplementedCoreRPCServer) GetPod(context.Context, *GetPodOptions) (*Pod, error) { return nil, status.Errorf(codes.Unimplemented, "method GetPod not implemented") } -func (*UnimplementedCoreRPCServer) ListPods(ctx context.Context, req *Empty) (*Pods, error) { +func (*UnimplementedCoreRPCServer) ListPods(context.Context, *Empty) (*Pods, error) { return nil, status.Errorf(codes.Unimplemented, "method ListPods not implemented") } -func (*UnimplementedCoreRPCServer) GetPodResource(ctx context.Context, req *GetPodOptions) (*PodResource, error) { +func (*UnimplementedCoreRPCServer) GetPodResource(context.Context, *GetPodOptions) (*PodResource, error) { return nil, status.Errorf(codes.Unimplemented, "method GetPodResource not implemented") } -func (*UnimplementedCoreRPCServer) AddNode(ctx context.Context, req *AddNodeOptions) (*Node, error) { +func (*UnimplementedCoreRPCServer) AddNode(context.Context, *AddNodeOptions) (*Node, error) { return nil, status.Errorf(codes.Unimplemented, "method AddNode not implemented") } -func (*UnimplementedCoreRPCServer) RemoveNode(ctx context.Context, req *RemoveNodeOptions) (*Empty, error) { +func (*UnimplementedCoreRPCServer) RemoveNode(context.Context, *RemoveNodeOptions) (*Empty, error) { return nil, status.Errorf(codes.Unimplemented, "method RemoveNode not implemented") } -func (*UnimplementedCoreRPCServer) ListPodNodes(ctx context.Context, req *ListNodesOptions) (*Nodes, error) { +func (*UnimplementedCoreRPCServer) ListPodNodes(context.Context, *ListNodesOptions) (*Nodes, error) { return nil, status.Errorf(codes.Unimplemented, "method ListPodNodes not implemented") } -func (*UnimplementedCoreRPCServer) GetNode(ctx context.Context, req *GetNodeOptions) (*Node, error) { +func (*UnimplementedCoreRPCServer) GetNode(context.Context, *GetNodeOptions) (*Node, error) { return nil, status.Errorf(codes.Unimplemented, "method GetNode not implemented") } -func (*UnimplementedCoreRPCServer) SetNode(ctx context.Context, req *SetNodeOptions) (*Node, error) { +func (*UnimplementedCoreRPCServer) SetNode(context.Context, *SetNodeOptions) (*Node, error) { return nil, status.Errorf(codes.Unimplemented, "method SetNode not implemented") } -func (*UnimplementedCoreRPCServer) GetNodeResource(ctx context.Context, req *GetNodeResourceOptions) (*NodeResource, error) { +func (*UnimplementedCoreRPCServer) GetNodeResource(context.Context, *GetNodeResourceOptions) (*NodeResource, error) { return nil, status.Errorf(codes.Unimplemented, "method GetNodeResource not implemented") } -func (*UnimplementedCoreRPCServer) GetContainer(ctx context.Context, req *ContainerID) (*Container, error) { +func (*UnimplementedCoreRPCServer) GetContainer(context.Context, *ContainerID) (*Container, error) { return nil, status.Errorf(codes.Unimplemented, "method GetContainer not implemented") } -func (*UnimplementedCoreRPCServer) GetContainers(ctx context.Context, req *ContainerIDs) (*Containers, error) { +func (*UnimplementedCoreRPCServer) GetContainers(context.Context, *ContainerIDs) (*Containers, error) { return nil, status.Errorf(codes.Unimplemented, "method GetContainers not implemented") } -func (*UnimplementedCoreRPCServer) ListContainers(req *ListContainersOptions, srv CoreRPC_ListContainersServer) error { +func (*UnimplementedCoreRPCServer) ListContainers(*ListContainersOptions, CoreRPC_ListContainersServer) error { return status.Errorf(codes.Unimplemented, "method ListContainers not implemented") } -func (*UnimplementedCoreRPCServer) ListNodeContainers(ctx context.Context, req *GetNodeOptions) (*Containers, error) { +func (*UnimplementedCoreRPCServer) ListNodeContainers(context.Context, *GetNodeOptions) (*Containers, error) { return nil, status.Errorf(codes.Unimplemented, "method ListNodeContainers not implemented") } -func (*UnimplementedCoreRPCServer) GetContainersStatus(ctx context.Context, req *ContainerIDs) (*ContainersStatus, error) { +func (*UnimplementedCoreRPCServer) GetContainersStatus(context.Context, *ContainerIDs) (*ContainersStatus, error) { return nil, status.Errorf(codes.Unimplemented, "method GetContainersStatus not implemented") } -func (*UnimplementedCoreRPCServer) SetContainersStatus(ctx context.Context, req *SetContainersStatusOptions) (*ContainersStatus, error) { +func (*UnimplementedCoreRPCServer) SetContainersStatus(context.Context, *SetContainersStatusOptions) (*ContainersStatus, error) { return nil, status.Errorf(codes.Unimplemented, "method SetContainersStatus not implemented") } -func (*UnimplementedCoreRPCServer) ContainerStatusStream(req *ContainerStatusStreamOptions, srv CoreRPC_ContainerStatusStreamServer) error { +func (*UnimplementedCoreRPCServer) ContainerStatusStream(*ContainerStatusStreamOptions, CoreRPC_ContainerStatusStreamServer) error { return status.Errorf(codes.Unimplemented, "method ContainerStatusStream not implemented") } -func (*UnimplementedCoreRPCServer) Copy(req *CopyOptions, srv CoreRPC_CopyServer) error { +func (*UnimplementedCoreRPCServer) Copy(*CopyOptions, CoreRPC_CopyServer) error { return status.Errorf(codes.Unimplemented, "method Copy not implemented") } -func (*UnimplementedCoreRPCServer) Send(req *SendOptions, srv CoreRPC_SendServer) error { +func (*UnimplementedCoreRPCServer) Send(*SendOptions, CoreRPC_SendServer) error { return status.Errorf(codes.Unimplemented, "method Send not implemented") } -func (*UnimplementedCoreRPCServer) BuildImage(req *BuildImageOptions, srv CoreRPC_BuildImageServer) error { +func (*UnimplementedCoreRPCServer) BuildImage(*BuildImageOptions, CoreRPC_BuildImageServer) error { return status.Errorf(codes.Unimplemented, "method BuildImage not implemented") } -func (*UnimplementedCoreRPCServer) CacheImage(req *CacheImageOptions, srv CoreRPC_CacheImageServer) error { +func (*UnimplementedCoreRPCServer) CacheImage(*CacheImageOptions, CoreRPC_CacheImageServer) error { return status.Errorf(codes.Unimplemented, "method CacheImage not implemented") } -func (*UnimplementedCoreRPCServer) RemoveImage(req *RemoveImageOptions, srv CoreRPC_RemoveImageServer) error { +func (*UnimplementedCoreRPCServer) RemoveImage(*RemoveImageOptions, CoreRPC_RemoveImageServer) error { return status.Errorf(codes.Unimplemented, "method RemoveImage not implemented") } -func (*UnimplementedCoreRPCServer) CreateContainer(req *DeployOptions, srv CoreRPC_CreateContainerServer) error { +func (*UnimplementedCoreRPCServer) CreateContainer(*DeployOptions, CoreRPC_CreateContainerServer) error { return status.Errorf(codes.Unimplemented, "method CreateContainer not implemented") } -func (*UnimplementedCoreRPCServer) ReplaceContainer(req *ReplaceOptions, srv CoreRPC_ReplaceContainerServer) error { +func (*UnimplementedCoreRPCServer) ReplaceContainer(*ReplaceOptions, CoreRPC_ReplaceContainerServer) error { return status.Errorf(codes.Unimplemented, "method ReplaceContainer not implemented") } -func (*UnimplementedCoreRPCServer) RemoveContainer(req *RemoveContainerOptions, srv CoreRPC_RemoveContainerServer) error { +func (*UnimplementedCoreRPCServer) RemoveContainer(*RemoveContainerOptions, CoreRPC_RemoveContainerServer) error { return status.Errorf(codes.Unimplemented, "method RemoveContainer not implemented") } -func (*UnimplementedCoreRPCServer) DissociateContainer(req *DissociateContainerOptions, srv CoreRPC_DissociateContainerServer) error { +func (*UnimplementedCoreRPCServer) DissociateContainer(*DissociateContainerOptions, CoreRPC_DissociateContainerServer) error { return status.Errorf(codes.Unimplemented, "method DissociateContainer not implemented") } -func (*UnimplementedCoreRPCServer) ControlContainer(req *ControlContainerOptions, srv CoreRPC_ControlContainerServer) error { +func (*UnimplementedCoreRPCServer) ControlContainer(*ControlContainerOptions, CoreRPC_ControlContainerServer) error { return status.Errorf(codes.Unimplemented, "method ControlContainer not implemented") } -func (*UnimplementedCoreRPCServer) ExecuteContainer(srv CoreRPC_ExecuteContainerServer) error { +func (*UnimplementedCoreRPCServer) ExecuteContainer(CoreRPC_ExecuteContainerServer) error { return status.Errorf(codes.Unimplemented, "method ExecuteContainer not implemented") } -func (*UnimplementedCoreRPCServer) ReallocResource(req *ReallocOptions, srv CoreRPC_ReallocResourceServer) error { - return status.Errorf(codes.Unimplemented, "method ReallocResource not implemented") +func (*UnimplementedCoreRPCServer) ReallocResource(context.Context, *ReallocOptions) (*ReallocResourceMessage, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReallocResource not implemented") } -func (*UnimplementedCoreRPCServer) LogStream(req *LogStreamOptions, srv CoreRPC_LogStreamServer) error { +func (*UnimplementedCoreRPCServer) LogStream(*LogStreamOptions, CoreRPC_LogStreamServer) error { return status.Errorf(codes.Unimplemented, "method LogStream not implemented") } -func (*UnimplementedCoreRPCServer) RunAndWait(srv CoreRPC_RunAndWaitServer) error { +func (*UnimplementedCoreRPCServer) RunAndWait(CoreRPC_RunAndWaitServer) error { return status.Errorf(codes.Unimplemented, "method RunAndWait not implemented") } @@ -6955,25 +9306,22 @@ func (x *coreRPCExecuteContainerServer) Recv() (*ExecuteContainerOptions, error) return m, nil } -func _CoreRPC_ReallocResource_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(ReallocOptions) - if err := stream.RecvMsg(m); err != nil { - return err +func _CoreRPC_ReallocResource_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReallocOptions) + if err := dec(in); err != nil { + return nil, err } - return srv.(CoreRPCServer).ReallocResource(m, &coreRPCReallocResourceServer{stream}) -} - -type CoreRPC_ReallocResourceServer interface { - Send(*ReallocResourceMessage) error - grpc.ServerStream -} - -type coreRPCReallocResourceServer struct { - grpc.ServerStream -} - -func (x *coreRPCReallocResourceServer) Send(m *ReallocResourceMessage) error { - return x.ServerStream.SendMsg(m) + if interceptor == nil { + return srv.(CoreRPCServer).ReallocResource(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pb.CoreRPC/ReallocResource", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(CoreRPCServer).ReallocResource(ctx, req.(*ReallocOptions)) + } + return interceptor(ctx, in, info, handler) } func _CoreRPC_LogStream_Handler(srv interface{}, stream grpc.ServerStream) error { @@ -7107,6 +9455,10 @@ var _CoreRPC_serviceDesc = grpc.ServiceDesc{ MethodName: "SetContainersStatus", Handler: _CoreRPC_SetContainersStatus_Handler, }, + { + MethodName: "ReallocResource", + Handler: _CoreRPC_ReallocResource_Handler, + }, }, Streams: []grpc.StreamDesc{ { @@ -7180,11 +9532,6 @@ var _CoreRPC_serviceDesc = grpc.ServiceDesc{ ServerStreams: true, ClientStreams: true, }, - { - StreamName: "ReallocResource", - Handler: _CoreRPC_ReallocResource_Handler, - ServerStreams: true, - }, { StreamName: "LogStream", Handler: _CoreRPC_LogStream_Handler, diff --git a/rpc/gen/core.proto b/rpc/gen/core.proto index 3ff60e8d6..d5fd92dc4 100644 --- a/rpc/gen/core.proto +++ b/rpc/gen/core.proto @@ -46,7 +46,7 @@ service CoreRPC { rpc DissociateContainer(DissociateContainerOptions) returns (stream DissociateContainerMessage) {}; rpc ControlContainer(ControlContainerOptions) returns (stream ControlContainerMessage) {}; rpc ExecuteContainer(stream ExecuteContainerOptions) returns (stream AttachContainerMessage) {}; - rpc ReallocResource(ReallocOptions) returns (stream ReallocResourceMessage) {}; + rpc ReallocResource(ReallocOptions) returns (ReallocResourceMessage) {}; rpc LogStream(LogStreamOptions) returns (stream LogStreamMessage) {}; rpc RunAndWait(stream RunAndWaitOptions) returns (stream AttachContainerMessage) {}; } @@ -181,22 +181,12 @@ message Container { string podname = 2; string nodename = 3; string name = 4; - map cpu = 5; - double quota = 6; - int64 memory = 7; - bool privileged = 8; - map labels = 9; - map publish = 10; - string image = 11; - int64 storage = 12; - ContainerStatus status = 13; - repeated string volumes = 14; - map volume_plan = 15; - double quota_request = 16; - int64 memory_request = 17; - int64 storage_request = 18; - repeated string volumes_request = 19; - map volume_plan_request = 20; + bool privileged = 5; + map labels = 6; + map publish = 7; + string image = 8; + ContainerStatus status = 9; + Resource resource = 10; } message ContainerStatus { @@ -260,21 +250,9 @@ enum TriOpt { } message ReallocOptions { - repeated string ids = 1; - double cpu = 2; - int64 memory = 3; - repeated string volumes = 4; - TriOpt bind_cpu_opt = 5; - TriOpt memory_limit_opt = 6; - - double cpu_request = 7; - double cpu_limit = 8; - int64 memory_request = 9; - int64 memory_limit = 10; - int64 storage_request = 11; - int64 storage_limit = 12; - repeated string volume_request = 13; - repeated string volume_limit = 14; + string id = 1; + TriOpt bind_cpu_opt = 2; + ResourceOptions resource_opts = 3; } message AddPodOptions { @@ -395,46 +373,67 @@ message EntrypointOptions { map sysctls = 10; } +message ResourceOptions { + double cpu_quota_limit = 1; + double cpu_quota_request = 2; + bool cpu_bind = 3; + int64 memory_limit = 4; + int64 memory_request = 5; + int64 storage_limit = 6; + int64 storage_request = 7; + repeated string volumes_limit = 8; + repeated string volumes_request = 9; +} + +message Resource { + double cpu_quota_limit = 1; + double cpu_quota_request = 2; + map cpu = 3; + int64 memory_limit = 4; + int64 memory_request = 5; + int64 storage_limit = 6; + int64 storage_request = 7; + repeated string volumes_limit = 8; + repeated string volumes_request = 9; + map volume_plan_limit = 10; + map volume_plan_request = 11; +} + +message Volume { + map volume = 1; +} + message DeployOptions { - string name = 1; - EntrypointOptions entrypoint = 2; - string podname = 3; - repeated string nodenames = 4; - string image = 5; - string extra_args = 6; - double cpu_quota = 7; - int64 memory = 8; - int32 count = 9; - repeated string env = 10; - repeated string dns = 11; - repeated string extra_hosts = 12; - repeated string volumes = 13; - map networks = 14; - string networkmode = 15; - string user = 16; - bool debug = 17; - bool openStdin = 18; - map labels = 19; - map nodelabels = 20; enum Strategy { AUTO = 0; FILL = 1; EACH = 2; GLOBAL = 3; } - Strategy deploy_strategy = 21; - map data = 22; - bool soft_limit = 23; - int32 nodes_limit = 24; - bool cpu_bind = 25; - bool ignore_hook = 26; - repeated string after_create = 27; - bytes raw_args = 28; - int64 storage = 29; - double cpu_quota_request = 30; // original cpu_quota denotes cpu_quota_limit - int64 memory_request = 31; // original memory denotes memory_limit - repeated string volumes_request = 32; // original volumes denotes volumes_limit - int64 storage_request = 33; // original storage denotes storage_limit + string name = 1; + EntrypointOptions entrypoint = 2; + string podname = 3; + repeated string nodenames = 4; + string image = 5; + string extra_args = 6; + int32 count = 7; + repeated string env = 8; + repeated string dns = 9; + repeated string extra_hosts = 10; + map networks = 11; + string networkmode = 12; + string user = 13; + bool debug = 14; + bool openStdin = 15; + map labels = 16; + map nodelabels = 17; + Strategy deploy_strategy = 18; + map data = 19; + int32 nodes_limit = 20; + bool ignore_hook = 21; + repeated string after_create = 22; + bytes raw_args = 23; + ResourceOptions resource_opts = 24; } message ReplaceOptions { @@ -487,10 +486,6 @@ message BuildImageMessage { ErrorDetail error_detail = 6; } -message Volume { - map volume = 1; -} - message CreateContainerMessage { string podname = 1; string nodename = 2; @@ -498,17 +493,9 @@ message CreateContainerMessage { string name = 4; string error = 5; bool success = 6; - map cpu = 7; - double quota = 8; - int64 memory = 9; - map publish = 10; - bytes hook = 11; - int64 storage = 12; - map volume_plan = 13; - double quota_request = 14; - int64 memory_request = 15; - int64 storage_request = 16; - map volume_plan_request = 17; + map publish = 7; + bytes hook = 8; + Resource resource = 9; } message ReplaceContainerMessage { @@ -542,8 +529,7 @@ message DissociateContainerMessage { } message ReallocResourceMessage { - string id = 1; - string error = 2; + string error = 1; } message CopyMessage { diff --git a/rpc/rpc.go b/rpc/rpc.go index 45fb29f63..a1f977bbe 100644 --- a/rpc/rpc.go +++ b/rpc/rpc.go @@ -651,56 +651,49 @@ func (v *Vibranium) ExecuteContainer(stream pb.CoreRPC_ExecuteContainerServer) ( } // ReallocResource realloc res for containers -func (v *Vibranium) ReallocResource(opts *pb.ReallocOptions, stream pb.CoreRPC_ReallocResourceServer) error { +func (v *Vibranium) ReallocResource(ctx context.Context, opts *pb.ReallocOptions) (msg *pb.ReallocResourceMessage, err error) { + defer func() { + errString := "" + if err != nil { + errString = err.Error() + } + msg = &pb.ReallocResourceMessage{Error: errString} + }() + v.taskAdd("ReallocResource", true) defer v.taskDone("ReallocResource", true) - ids := opts.GetIds() - if len(ids) == 0 { - return types.ErrNoContainerIDs - } - vbs, err := types.NewVolumeBindings(opts.Volumes) - if err != nil { - return err + if opts.Id == "" { + return msg, types.ErrNoContainerIDs } - vbsRequest, err := types.NewVolumeBindings(opts.VolumeRequest) + + vbsRequest, err := types.NewVolumeBindings(opts.ResourceOpts.VolumesRequest) if err != nil { - return err + return msg, err } - vbsLimit, err := types.NewVolumeBindings(opts.VolumeLimit) + + vbsLimit, err := types.NewVolumeBindings(opts.ResourceOpts.VolumesLimit) if err != nil { - return err + return msg, err } //这里不能让 client 打断 remove - ch, err := v.cluster.ReallocResource( - stream.Context(), - &types.ReallocOptions{IDs: ids, - CPU: opts.Cpu, - BindCPUOpt: types.TriOptions(opts.BindCpuOpt), - Memory: opts.Memory, - MemoryLimitOpt: types.TriOptions(opts.MemoryLimitOpt), - Volumes: vbs, - - CPURequest: opts.CpuRequest, - CPULimit: opts.CpuLimit, - MemoryRequest: opts.MemoryRequest, - MemoryLimit: opts.MemoryLimit, - VolumeRequest: vbsRequest, - VolumeLimit: vbsLimit, - StorageRequest: opts.StorageRequest, - StorageLimit: opts.StorageLimit, + return msg, v.cluster.ReallocResource( + ctx, + &types.ReallocOptions{ + ID: opts.Id, + CPUBindOpts: types.TriOptions(opts.BindCpuOpt), + ResourceOpts: types.ResourceOptions{ + CPUQuotaRequest: opts.ResourceOpts.CpuQuotaRequest, + CPUQuotaLimit: opts.ResourceOpts.CpuQuotaLimit, + MemoryRequest: opts.ResourceOpts.MemoryRequest, + MemoryLimit: opts.ResourceOpts.MemoryLimit, + VolumeRequest: vbsRequest, + VolumeLimit: vbsLimit, + StorageRequest: opts.ResourceOpts.StorageRequest, + StorageLimit: opts.ResourceOpts.StorageLimit, + }, }, ) - if err != nil { - return err - } - - for m := range ch { - if err = stream.Send(toRPCReallocResourceMessage(m)); err != nil { - v.logUnsentMessages("ReallocResource", m) - } - } - return err } // LogStream get container logs diff --git a/rpc/transform.go b/rpc/transform.go index 249163a8f..99ad117f5 100644 --- a/rpc/transform.go +++ b/rpc/transform.go @@ -268,24 +268,36 @@ func toCoreDeployOptions(d *pb.DeployOptions) (*types.DeployOptions, error) { entry.Hook.Force = entrypoint.Hook.Force } - vbs, err := types.NewVolumeBindings(d.Volumes) - if err != nil { - return nil, err + var err error + data := map[string]types.ReaderManager{} + for filename, bs := range d.Data { + if data[filename], err = types.NewReaderManager(bytes.NewBuffer(bs)); err != nil { + return nil, err + } } - vbsRequest, err := types.NewVolumeBindings(d.VolumesRequest) + vbsLimit, err := types.NewVolumeBindings(d.ResourceOpts.VolumesLimit) if err != nil { return nil, err } - data := map[string]types.ReaderManager{} - for filename, bs := range d.Data { - if data[filename], err = types.NewReaderManager(bytes.NewBuffer(bs)); err != nil { - return nil, err - } + vbsRequest, err := types.NewVolumeBindings(d.ResourceOpts.VolumesRequest) + if err != nil { + return nil, err } return &types.DeployOptions{ + ResourceOpts: types.ResourceOptions{ + CPUQuotaRequest: d.ResourceOpts.CpuQuotaRequest, + CPUQuotaLimit: d.ResourceOpts.CpuQuotaLimit, + CPUBind: d.ResourceOpts.CpuBind, + MemoryRequest: d.ResourceOpts.MemoryRequest, + MemoryLimit: d.ResourceOpts.MemoryLimit, + VolumeRequest: vbsRequest, + VolumeLimit: vbsLimit, + StorageRequest: d.ResourceOpts.StorageRequest, + StorageLimit: d.ResourceOpts.StorageLimit, + }, Name: d.Name, Entrypoint: entry, Podname: d.Podname, @@ -304,60 +316,37 @@ func toCoreDeployOptions(d *pb.DeployOptions) (*types.DeployOptions, error) { Labels: d.Labels, NodeLabels: d.Nodelabels, DeployStrategy: d.DeployStrategy.String(), - SoftLimit: d.SoftLimit, NodesLimit: int(d.NodesLimit), IgnoreHook: d.IgnoreHook, AfterCreate: d.AfterCreate, RawArgs: d.RawArgs, Data: data, - RawResourceOptions: types.RawResourceOptions{ - CPURequest: d.CpuQuotaRequest, - CPULimit: d.CpuQuota, - CPUBind: d.CpuBind, - MemoryRequest: d.MemoryRequest, - MemoryLimit: d.Memory, - MemorySoft: d.SoftLimit, - VolumeRequest: vbsRequest, - VolumeLimit: vbs, - StorageRequest: d.StorageRequest, - StorageLimit: d.Storage, - }, }, nil } -func toRPCVolumePlan(v types.VolumePlan) map[string]*pb.Volume { - if v == nil { - return nil - } - - msg := map[string]*pb.Volume{} - for vb, volume := range v { - msg[vb.ToString(false)] = &pb.Volume{Volume: volume} - } - return msg -} - func toRPCCreateContainerMessage(c *types.CreateContainerMessage) *pb.CreateContainerMessage { if c == nil { return nil } msg := &pb.CreateContainerMessage{ - Podname: c.Podname, - Nodename: c.Nodename, - Id: c.ContainerID, - Name: c.ContainerName, - Success: c.Error == nil, - Publish: utils.EncodePublishInfo(c.Publish), - Hook: utils.MergeHookOutputs(c.Hook), - Cpu: toRPCCPUMap(c.CPULimit), - Quota: c.CPUQuotaLimit, - QuotaRequest: c.CPUQuotaRequest, - Memory: c.MemoryLimit, - MemoryRequest: c.MemoryRequest, - Storage: c.StorageLimit, - StorageRequest: c.StorageRequest, - VolumePlan: toRPCVolumePlan(c.VolumePlanLimit), - VolumePlanRequest: toRPCVolumePlan(c.VolumePlanRequest), + Podname: c.Podname, + Nodename: c.Nodename, + Id: c.ContainerID, + Name: c.ContainerName, + Success: c.Error == nil, + Publish: utils.EncodePublishInfo(c.Publish), + Hook: utils.MergeHookOutputs(c.Hook), + Resource: &pb.Resource{ + CpuQuotaLimit: c.CPUQuotaLimit, + CpuQuotaRequest: c.CPUQuotaRequest, + Cpu: toRPCCPUMap(c.CPU), + MemoryLimit: c.MemoryLimit, + MemoryRequest: c.MemoryRequest, + StorageLimit: c.StorageLimit, + StorageRequest: c.StorageRequest, + VolumesLimit: c.VolumeLimit.ToStringSlice(false, false), + VolumesRequest: c.VolumeRequest.ToStringSlice(false, false), + }, } if c.Error != nil { msg.Error = c.Error.Error() @@ -404,16 +393,6 @@ func toRPCControlContainerMessage(c *types.ControlContainerMessage) *pb.ControlC return r } -func toRPCReallocResourceMessage(r *types.ReallocResourceMessage) *pb.ReallocResourceMessage { - resp := &pb.ReallocResourceMessage{ - Id: r.ContainerID, - } - if r.Error != nil { - resp.Error = r.Error.Error() - } - return resp -} - func toRPCRemoveContainerMessage(r *types.RemoveContainerMessage) *pb.RemoveContainerMessage { if r == nil { return nil @@ -493,31 +472,44 @@ func toRPCContainer(_ context.Context, c *types.Container) (*pb.Container, error utils.MakePublishInfo(c.StatusMeta.Networks, meta.Publish), ) } - cpu := toRPCCPUMap(c.CPURequest) return &pb.Container{ - Id: c.ID, - Podname: c.Podname, - Nodename: c.Nodename, - Name: c.Name, - Cpu: cpu, - Privileged: c.Privileged, - Publish: publish, - Image: c.Image, - Labels: c.Labels, - Status: toRPCContainerStatus(c.StatusMeta), - Quota: c.QuotaLimit, - QuotaRequest: c.QuotaRequest, - Memory: c.MemoryLimit, - MemoryRequest: c.MemoryRequest, - Storage: c.StorageLimit, - StorageRequest: c.StorageRequest, - Volumes: c.VolumeLimit.ToStringSlice(false, false), - VolumesRequest: c.VolumeRequest.ToStringSlice(false, false), - VolumePlan: toRPCVolumePlan(c.VolumePlanLimit), - VolumePlanRequest: toRPCVolumePlan(c.VolumePlanRequest), + Id: c.ID, + Podname: c.Podname, + Nodename: c.Nodename, + Name: c.Name, + Privileged: c.Privileged, + Publish: publish, + Image: c.Image, + Labels: c.Labels, + Status: toRPCContainerStatus(c.StatusMeta), + Resource: &pb.Resource{ + CpuQuotaLimit: c.CPUQuotaLimit, + CpuQuotaRequest: c.CPUQuotaRequest, + Cpu: toRPCCPUMap(c.CPU), + MemoryLimit: c.MemoryLimit, + MemoryRequest: c.MemoryRequest, + StorageLimit: c.StorageLimit, + StorageRequest: c.StorageRequest, + VolumesLimit: c.VolumeLimit.ToStringSlice(false, false), + VolumesRequest: c.VolumeRequest.ToStringSlice(false, false), + VolumePlanLimit: toRPCVolumePlan(c.VolumePlanLimit), + VolumePlanRequest: toRPCVolumePlan(c.VolumePlanRequest), + }, }, nil } +func toRPCVolumePlan(v types.VolumePlan) map[string]*pb.Volume { + if v == nil { + return nil + } + + msg := map[string]*pb.Volume{} + for vb, volume := range v { + msg[vb.ToString(false)] = &pb.Volume{Volume: volume} + } + return msg +} + func toRPCLogStreamMessage(msg *types.LogStreamMessage) *pb.LogStreamMessage { r := &pb.LogStreamMessage{ Id: msg.ID, diff --git a/scheduler/complex/potassium_test.go b/scheduler/complex/potassium_test.go index 4a604c0b6..902a5355f 100644 --- a/scheduler/complex/potassium_test.go +++ b/scheduler/complex/potassium_test.go @@ -192,7 +192,7 @@ func getNodeMapFromNodesInfo(nodesInfo []types.NodeInfo) map[string]*types.Node return nodeMap } -func getInfosFromNodesInfo(nodesInfo []types.NodeInfo, planMap map[types.ResourceType]resourcetypes.ResourcePlans) (strategyInfos []strategy.Info) { +func getInfosFromNodesInfo(nodesInfo []types.NodeInfo, planMap []resourcetypes.ResourcePlans) (strategyInfos []strategy.Info) { for _, nodeInfo := range nodesInfo { capacity := math.MaxInt32 for _, v := range planMap { @@ -230,12 +230,12 @@ func newDeployOptions(need int, each bool) *types.DeployOptions { } func SelectCPUNodes(k *Potassium, nodesInfo []types.NodeInfo, quota float64, memory int64, need int, each bool) (map[string][]types.CPUMap, map[string]types.CPUMap, error) { - rrs, err := resources.NewResourceRequirements(types.RawResourceOptions{CPULimit: quota, MemoryLimit: memory, CPUBind: true}) + rrs, err := resources.MakeRequests(types.ResourceOptions{CPUQuotaLimit: quota, MemoryLimit: memory, CPUBind: true}) if err != nil { return nil, nil, err } nodeMap := getNodeMapFromNodesInfo(nodesInfo) - planMap, total, sType, err := resources.SelectNodes(rrs, nodeMap) + sType, total, planMap, err := resources.SelectNodesByResourceRequests(rrs, nodeMap) if err != nil { return nil, nil, err } @@ -259,11 +259,11 @@ func SelectCPUNodes(k *Potassium, nodesInfo []types.NodeInfo, quota float64, mem } func SelectMemoryNodes(k *Potassium, nodesInfo []types.NodeInfo, rate float64, memory int64, need int, each bool) ([]types.NodeInfo, error) { - rrs, err := resources.NewResourceRequirements(types.RawResourceOptions{CPULimit: rate, MemoryLimit: memory}) + rrs, err := resources.MakeRequests(types.ResourceOptions{CPUQuotaLimit: rate, MemoryLimit: memory}) if err != nil { return nil, err } - planMap, total, sType, err := resources.SelectNodes(rrs, getNodeMapFromNodesInfo(nodesInfo)) + sType, total, planMap, err := resources.SelectNodesByResourceRequests(rrs, getNodeMapFromNodesInfo(nodesInfo)) if err != nil { return nil, err } @@ -1210,11 +1210,11 @@ func TestSelectStorageNodesSequence(t *testing.T) { } func SelectStorageNodes(k *Potassium, nodesInfo []types.NodeInfo, storage int64, need int, each bool) ([]types.NodeInfo, error) { - rrs, err := resources.NewResourceRequirements(types.RawResourceOptions{StorageLimit: storage}) + rrs, err := resources.MakeRequests(types.ResourceOptions{StorageLimit: storage}) if err != nil { return nil, err } - planMap, total, sType, err := resources.SelectNodes(rrs, getNodeMapFromNodesInfo(nodesInfo)) + sType, total, planMap, err := resources.SelectNodesByResourceRequests(rrs, getNodeMapFromNodesInfo(nodesInfo)) if err != nil { return nil, err } @@ -1236,12 +1236,12 @@ func SelectStorageNodes(k *Potassium, nodesInfo []types.NodeInfo, storage int64, } func SelectVolumeNodes(k *Potassium, nodesInfo []types.NodeInfo, volumes []string, need int, each bool) (map[string][]types.VolumePlan, map[string]types.VolumeMap, error) { - rrs, err := resources.NewResourceRequirements(types.RawResourceOptions{VolumeLimit: types.MustToVolumeBindings(volumes)}) + rrs, err := resources.MakeRequests(types.ResourceOptions{VolumeLimit: types.MustToVolumeBindings(volumes)}) if err != nil { return nil, nil, err } nodeMap := getNodeMapFromNodesInfo(nodesInfo) - planMap, total, sType, err := resources.SelectNodes(rrs, nodeMap) + sType, total, planMap, err := resources.SelectNodesByResourceRequests(rrs, nodeMap) if err != nil { return nil, nil, err } @@ -1255,7 +1255,7 @@ func SelectVolumeNodes(k *Potassium, nodesInfo []types.NodeInfo, volumes []strin for nodename, deploy := range deployMap { for _, plan := range planMap { if volumePlan, ok := plan.(volume.ResourcePlans); ok { - result[nodename] = volumePlan.PlanReq[nodename] + result[nodename] = volumePlan.GetPlan(nodename) plan.ApplyChangesOnNode(nodeMap[nodename], utils.Range(deploy)...) changed[nodename] = nodeMap[nodename].Volume } diff --git a/store/etcdv3/node.go b/store/etcdv3/node.go index 05e64d3a1..3fbff2b18 100644 --- a/store/etcdv3/node.go +++ b/store/etcdv3/node.go @@ -140,32 +140,15 @@ func (m *Mercury) UpdateNodes(ctx context.Context, nodes ...*types.Node) error { } // UpdateNodeResource update cpu and memory on a node, either add or subtract -func (m *Mercury) UpdateNodeResource(ctx context.Context, node *types.Node, cpu types.CPUMap, quota float64, memory, storage int64, volume types.VolumeMap, action string) error { +func (m *Mercury) UpdateNodeResource(ctx context.Context, node *types.Node, resource *types.ResourceMeta, action string) error { switch action { case store.ActionIncr: - node.CPU.Add(cpu) - node.SetCPUUsed(quota, types.DecrUsage) - node.Volume.Add(volume) - node.SetVolumeUsed(volume.Total(), types.DecrUsage) - node.MemCap += memory - node.StorageCap += storage - if nodeID := node.GetNUMANode(cpu); nodeID != "" { - node.IncrNUMANodeMemory(nodeID, memory) - } + node.RecycleResources(resource) case store.ActionDecr: - node.CPU.Sub(cpu) - node.SetCPUUsed(quota, types.IncrUsage) - node.Volume.Sub(volume) - node.SetVolumeUsed(volume.Total(), types.IncrUsage) - node.MemCap -= memory - node.StorageCap -= storage - if nodeID := node.GetNUMANode(cpu); nodeID != "" { - node.DecrNUMANodeMemory(nodeID, memory) - } + node.PreserveResources(resource) default: return types.ErrUnknownControlType } - go metrics.Client.SendNodeInfo(node) return m.UpdateNodes(ctx, node) } diff --git a/store/etcdv3/node_test.go b/store/etcdv3/node_test.go index e44fecd7a..a4d556315 100644 --- a/store/etcdv3/node_test.go +++ b/store/etcdv3/node_test.go @@ -205,7 +205,7 @@ func TestUpdateNodeResource(t *testing.T) { node, err := m.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", 1, 100, 100000, 100000, map[string]string{"x": "y"}, map[string]string{"0": "0"}, map[string]int64{"0": 100}, nil) assert.NoError(t, err) assert.Equal(t, node.Name, "test") - assert.Error(t, m.UpdateNodeResource(ctx, node, nil, 0, 0, 0, nil, "wtf")) - assert.NoError(t, m.UpdateNodeResource(ctx, node, map[string]int64{"0": 100}, 0, 0, 0, nil, store.ActionIncr)) - assert.NoError(t, m.UpdateNodeResource(ctx, node, map[string]int64{"0": 100}, 0, 0, 0, nil, store.ActionDecr)) + assert.Error(t, m.UpdateNodeResource(ctx, node, nil, "wtf")) + assert.NoError(t, m.UpdateNodeResource(ctx, node, &types.ResourceMeta{CPU: map[string]int64{"0": 100}}, store.ActionIncr)) + assert.NoError(t, m.UpdateNodeResource(ctx, node, &types.ResourceMeta{CPU: map[string]int64{"0": 100}}, store.ActionDecr)) } diff --git a/store/mocks/Store.go b/store/mocks/Store.go index 89d234799..a14671424 100644 --- a/store/mocks/Store.go +++ b/store/mocks/Store.go @@ -1,19 +1,14 @@ -// Code generated by mockery v2.0.0-alpha.2. DO NOT EDIT. +// Code generated by mockery v1.0.0. DO NOT EDIT. package mocks -import ( - context "context" +import context "context" +import lock "github.com/projecteru2/core/lock" +import mock "github.com/stretchr/testify/mock" - lock "github.com/projecteru2/core/lock" - mock "github.com/stretchr/testify/mock" - - strategy "github.com/projecteru2/core/strategy" - - time "time" - - types "github.com/projecteru2/core/types" -) +import strategy "github.com/projecteru2/core/strategy" +import time "time" +import types "github.com/projecteru2/core/types" // Store is an autogenerated mock type for the Store type type Store struct { @@ -517,13 +512,13 @@ func (_m *Store) UpdateContainer(ctx context.Context, container *types.Container return r0 } -// UpdateNodeResource provides a mock function with given fields: ctx, node, cpu, quota, memory, storage, volume, action -func (_m *Store) UpdateNodeResource(ctx context.Context, node *types.Node, cpu types.ResourceMap, quota float64, memory int64, storage int64, volume types.ResourceMap, action string) error { - ret := _m.Called(ctx, node, cpu, quota, memory, storage, volume, action) +// UpdateNodeResource provides a mock function with given fields: ctx, node, resource, action +func (_m *Store) UpdateNodeResource(ctx context.Context, node *types.Node, resource *types.ResourceMeta, action string) error { + ret := _m.Called(ctx, node, resource, action) var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *types.Node, types.ResourceMap, float64, int64, int64, types.ResourceMap, string) error); ok { - r0 = rf(ctx, node, cpu, quota, memory, storage, volume, action) + if rf, ok := ret.Get(0).(func(context.Context, *types.Node, *types.ResourceMeta, string) error); ok { + r0 = rf(ctx, node, resource, action) } else { r0 = ret.Error(0) } diff --git a/store/store.go b/store/store.go index a27141f7b..48a422226 100644 --- a/store/store.go +++ b/store/store.go @@ -36,7 +36,7 @@ type Store interface { GetNodes(ctx context.Context, nodenames []string) ([]*types.Node, error) GetNodesByPod(ctx context.Context, podname string, labels map[string]string, all bool) ([]*types.Node, error) UpdateNodes(context.Context, ...*types.Node) error - UpdateNodeResource(ctx context.Context, node *types.Node, cpu types.CPUMap, quota float64, memory, storage int64, volume types.VolumeMap, action string) error + UpdateNodeResource(ctx context.Context, node *types.Node, resource *types.ResourceMeta, action string) error // container AddContainer(ctx context.Context, container *types.Container) error diff --git a/strategy/strategy.go b/strategy/strategy.go index a3c95c513..672104be6 100644 --- a/strategy/strategy.go +++ b/strategy/strategy.go @@ -58,17 +58,18 @@ type Info struct { } // NewInfos . -func NewInfos(rrs resourcetypes.ResourceRequirements, nodeMap map[string]*types.Node, planMap map[types.ResourceType]resourcetypes.ResourcePlans) (strategyInfos []Info) { - for nodeName, node := range nodeMap { +// TODO strange name, need to revise +func NewInfos(resourceRequests resourcetypes.ResourceRequests, nodeMap map[string]*types.Node, plans []resourcetypes.ResourcePlans) (strategyInfos []Info) { + for nodename, node := range nodeMap { rates := make(map[types.ResourceType]float64) - for _, rr := range rrs { - rates[rr.Type()] = rr.Rate(*node) + for _, resourceRequest := range resourceRequests { + rates[resourceRequest.Type()] = resourceRequest.Rate(*node) } capacity := math.MaxInt32 - for _, plan := range planMap { - if plan.Capacity()[nodeName] < capacity { - capacity = plan.Capacity()[nodeName] + for _, plan := range plans { + if plan.Capacity()[nodename] < capacity { + capacity = plan.Capacity()[nodename] } } if capacity <= 0 { @@ -76,7 +77,7 @@ func NewInfos(rrs resourcetypes.ResourceRequirements, nodeMap map[string]*types. } strategyInfos = append(strategyInfos, Info{ - Nodename: nodeName, + Nodename: nodename, Rates: rates, Usages: node.ResourceUsages(), Capacity: capacity, diff --git a/strategy/strategy_test.go b/strategy/strategy_test.go index 9e3eb3603..145c860ba 100644 --- a/strategy/strategy_test.go +++ b/strategy/strategy_test.go @@ -90,7 +90,7 @@ func TestDeploy(t *testing.T) { } func TestNewInfos(t *testing.T) { - rrs, err := resources.NewResourceRequirements(types.RawResourceOptions{}) + rrs, err := resources.MakeRequests(types.ResourceOptions{}) assert.Nil(t, err) nodeMap := map[string]*types.Node{ "node1": {}, @@ -98,6 +98,6 @@ func TestNewInfos(t *testing.T) { } mockPlan := &resourcetypesmocks.ResourcePlans{} mockPlan.On("Capacity").Return(map[string]int{"node1": 1}) - planMap := map[types.ResourceType]resourcetypes.ResourcePlans{1: mockPlan} - NewInfos(rrs, nodeMap, planMap) + plans := []resourcetypes.ResourcePlans{mockPlan} + NewInfos(rrs, nodeMap, plans) } diff --git a/types/container.go b/types/container.go index a84f8b724..e34822b45 100644 --- a/types/container.go +++ b/types/container.go @@ -2,7 +2,6 @@ package types import ( "context" - "encoding/json" engine "github.com/projecteru2/core/engine" enginetypes "github.com/projecteru2/core/engine/types" @@ -28,55 +27,20 @@ type LabelMeta struct { // only relationship with pod and node is stored // if you wanna get realtime information, use Inspect method type Container struct { - ID string `json:"id"` - Name string `json:"name"` - Podname string `json:"podname"` - Nodename string `json:"nodename"` - CPURequest CPUMap `json:"cpu"` - CPULimit CPUMap `json:"cpu_limit"` - QuotaRequest float64 `json:"quota"` - QuotaLimit float64 `json:"quota_limit"` - MemoryRequest int64 `json:"memory"` - MemoryLimit int64 `json:"memory_limit"` - StorageRequest int64 `json:"storage"` - StorageLimit int64 `json:"storage_limit"` - Hook *Hook `json:"hook"` - Privileged bool `json:"privileged"` - SoftLimit bool `json:"softlimit"` - User string `json:"user"` - Env []string `json:"env"` - Image string `json:"image"` - VolumeRequest VolumeBindings `json:"volumes"` - VolumePlanRequest VolumePlan `json:"volume_plan"` - VolumeLimit VolumeBindings `json:"volumes_limit"` - VolumePlanLimit VolumePlan `json:"volume_plan_limit"` - Labels map[string]string `json:"labels"` - StatusMeta *StatusMeta `json:"-"` - Engine engine.API `json:"-"` - ResourceSubdivisible bool `json:"resource_subdivisible"` // to tell apart from existing container metas -} - -type container Container - -// UnmarshalJSON makes compatible for old resource fields -func (c *Container) UnmarshalJSON(b []byte) error { - cc := &container{} - if err := json.Unmarshal(b, cc); err != nil { - return err - } - - *c = Container(*cc) - - // existing container meta - if !c.ResourceSubdivisible { - c.QuotaLimit = c.QuotaRequest - c.CPULimit = c.CPURequest - c.StorageLimit = c.StorageRequest - c.MemoryLimit = c.MemoryRequest - c.VolumeLimit = c.VolumeRequest - c.VolumePlanLimit = c.VolumePlanRequest - } - return nil + ResourceMeta + ID string `json:"id"` + Name string `json:"name"` + Podname string `json:"podname"` + Nodename string `json:"nodename"` + Hook *Hook `json:"hook"` + Privileged bool `json:"privileged"` + SoftLimit bool `json:"softlimit"` + User string `json:"user"` + Env []string `json:"env"` + Image string `json:"image"` + Labels map[string]string `json:"labels"` + StatusMeta *StatusMeta `json:"-"` + Engine engine.API `json:"-"` } // Inspect a container diff --git a/types/container_test.go b/types/container_test.go index af23cf3a4..54c8fd6f9 100644 --- a/types/container_test.go +++ b/types/container_test.go @@ -2,7 +2,6 @@ package types import ( "context" - "encoding/json" "testing" "github.com/stretchr/testify/assert" @@ -49,14 +48,3 @@ func TestContainerControl(t *testing.T) { err = c.Remove(ctx, true) assert.NoError(t, err) } - -func TestContainerUnmarshal(t *testing.T) { - c := Container{QuotaRequest: 1} - b, err := json.Marshal(c) - assert.Nil(t, err) - c1 := Container{} - err = json.Unmarshal(b, &c1) - assert.Nil(t, err) - assert.EqualValues(t, 1, c1.QuotaLimit) - assert.EqualValues(t, 1, c1.QuotaRequest) -} diff --git a/types/message.go b/types/message.go index 9c75b3e18..9ba599afa 100644 --- a/types/message.go +++ b/types/message.go @@ -74,6 +74,7 @@ type ControlContainerMessage struct { // CreateContainerMessage for create message type CreateContainerMessage struct { + ResourceMeta Podname string Nodename string ContainerID string @@ -81,30 +82,6 @@ type CreateContainerMessage struct { Error error Publish map[string][]string Hook []*bytes.Buffer - Resources -} - -// Resources . -type Resources struct { - CPURequest CPUMap - CPULimit CPUMap - CPUQuotaRequest float64 - CPUQuotaLimit float64 - CPUBind bool - NUMANode string - - MemoryRequest int64 - MemoryLimit int64 - MemorySoftLimit bool - - VolumeRequest VolumeBindings - VolumeLimit VolumeBindings - VolumePlanRequest VolumePlan - VolumePlanLimit VolumePlan - VolumeChanged bool - - StorageRequest int64 - StorageLimit int64 } // ReplaceContainerMessage for replace method @@ -128,7 +105,6 @@ type PullImageMessage struct { // ReallocResourceMessage for realloc resource type ReallocResourceMessage struct { ContainerID string - Error error } // LogStreamMessage for log stream diff --git a/types/node.go b/types/node.go index 3c7dbf334..5385af221 100644 --- a/types/node.go +++ b/types/node.go @@ -2,8 +2,6 @@ package types import ( "context" - "encoding/json" - "sort" "math" @@ -20,175 +18,6 @@ const ( AUTO = "AUTO" ) -// ResourceMap is cpu core map -// ResourceMap {["0"]10000, ["1"]10000} -type ResourceMap map[string]int64 - -// Total show total cpu -// Total quotas -func (c ResourceMap) Total() int64 { - var count int64 - for _, value := range c { - count += value - } - return count -} - -// Add return cpu -func (c ResourceMap) Add(q ResourceMap) { - for label, value := range q { - if _, ok := c[label]; !ok { - c[label] = value - } else { - c[label] += value - } - } -} - -// Sub decrease cpus -func (c ResourceMap) Sub(q ResourceMap) { - for label, value := range q { - if _, ok := c[label]; ok { - c[label] -= value - } - } -} - -// CPUMap is cpu core map -// CPUMap {["0"]10000, ["1"]10000} -type CPUMap = ResourceMap - -// VolumeMap is volume map -// VolumeMap {["/data1"]1073741824, ["/data2"]1048576} -type VolumeMap = ResourceMap - -// GetResourceID returns device name such as "/sda0" -// GetResourceID only works for VolumeMap with single key -func (c VolumeMap) GetResourceID() (key string) { - for k := range c { - key = k - break - } - return -} - -// GetRation returns scheduled size from device -// GetRation only works for VolumeMap with single key -func (c VolumeMap) GetRation() int64 { - return c[c.GetResourceID()] -} - -// SplitByUsed . -func (c VolumeMap) SplitByUsed(init VolumeMap) (VolumeMap, VolumeMap) { - used := VolumeMap{} - unused := VolumeMap{} - for mountDir, freeSpace := range c { - vmap := used - if init[mountDir] == freeSpace { - vmap = unused - } - vmap.Add(VolumeMap{mountDir: freeSpace}) - } - return used, unused -} - -// VolumePlan is map from volume string to volumeMap: {"AUTO:/data:rw:100": VolumeMap{"/sda1": 100}} -type VolumePlan map[VolumeBinding]VolumeMap - -// MakeVolumePlan creates VolumePlan pointer by volume strings and scheduled VolumeMaps -func MakeVolumePlan(vbs VolumeBindings, distribution []VolumeMap) VolumePlan { - sort.Slice(vbs, func(i, j int) bool { return vbs[i].SizeInBytes < vbs[j].SizeInBytes }) - sort.Slice(distribution, func(i, j int) bool { return distribution[i].GetRation() < distribution[j].GetRation() }) - - volumePlan := VolumePlan{} - for idx, vb := range vbs { - if vb.RequireSchedule() { - volumePlan[*vb] = distribution[idx] - } - } - return volumePlan -} - -// UnmarshalJSON . -func (p *VolumePlan) UnmarshalJSON(b []byte) (err error) { - if *p == nil { - *p = VolumePlan{} - } - plan := map[string]VolumeMap{} - if err = json.Unmarshal(b, &plan); err != nil { - return err - } - for volume, vmap := range plan { - vb, err := NewVolumeBinding(volume) - if err != nil { - return err - } - (*p)[*vb] = vmap - } - return -} - -// MarshalJSON . -func (p VolumePlan) MarshalJSON() ([]byte, error) { - plan := map[string]VolumeMap{} - for vb, vmap := range p { - plan[vb.ToString(false)] = vmap - } - return json.Marshal(plan) -} - -// ToLiteral returns literal VolumePlan -func (p VolumePlan) ToLiteral() map[string]map[string]int64 { - plan := map[string]map[string]int64{} - for vb, volumeMap := range p { - plan[vb.ToString(false)] = volumeMap - } - return plan -} - -// IntoVolumeMap Merge return one VolumeMap with all in VolumePlan added -func (p VolumePlan) IntoVolumeMap() VolumeMap { - volumeMap := VolumeMap{} - for _, v := range p { - volumeMap.Add(v) - } - return volumeMap -} - -// GetVolumeMap looks up VolumeMap according to volume destination directory -func (p VolumePlan) GetVolumeMap(vb *VolumeBinding) (volMap VolumeMap, volume VolumeBinding) { - for volume, volMap := range p { - if vb.Destination == volume.Destination { - return volMap, volume - } - } - return -} - -// Compatible return true if new bindings stick to the old bindings -func (p VolumePlan) Compatible(oldPlan VolumePlan) bool { - for volume, oldBinding := range oldPlan { - newBinding, _ := p.GetVolumeMap(&volume) // nolint - // newBinding is ok to be nil when reallocing requires less volumes than before - if newBinding != nil && newBinding.GetResourceID() != oldBinding.GetResourceID() { - // unlimited binding, modify binding source - if newBinding.GetRation() == 0 { - //p[v] = VolumeMap{oldBinding.GetResourceID(): 0} - continue - } - return false - } - } - return true -} - -// Merge . -func (p VolumePlan) Merge(p2 VolumePlan) { - for vb, vm := range p2 { - p[vb] = vm - } -} - // NUMA define NUMA cpuID->nodeID type NUMA map[string]string @@ -328,6 +157,34 @@ func (n *Node) ResourceUsages() map[ResourceType]float64 { } } +// RecycleResources . +// TODO need tests +func (n *Node) RecycleResources(resource *ResourceMeta) { + n.CPU.Add(resource.CPU) + n.SetCPUUsed(resource.CPUQuotaRequest, DecrUsage) + n.Volume.Add(resource.VolumePlanRequest.IntoVolumeMap()) + n.SetVolumeUsed(resource.VolumePlanRequest.IntoVolumeMap().Total(), DecrUsage) + n.MemCap += resource.MemoryRequest + n.StorageCap += resource.StorageRequest + if resource.NUMANode != "" { + n.IncrNUMANodeMemory(resource.NUMANode, resource.MemoryRequest) + } +} + +// PreserveResources . +// TODO need tests +func (n *Node) PreserveResources(resource *ResourceMeta) { + n.CPU.Sub(resource.CPU) + n.SetCPUUsed(resource.CPUQuotaRequest, IncrUsage) + n.Volume.Sub(resource.VolumePlanRequest.IntoVolumeMap()) + n.SetVolumeUsed(resource.VolumePlanRequest.IntoVolumeMap().Total(), IncrUsage) + n.MemCap -= resource.MemoryRequest + n.StorageCap -= resource.StorageRequest + if resource.NUMANode != "" { + n.DecrNUMANodeMemory(resource.NUMANode, resource.MemoryRequest) + } +} + // NodeInfo for deploy type NodeInfo struct { Name string diff --git a/types/node_test.go b/types/node_test.go index 79fe81b65..f7bb27912 100644 --- a/types/node_test.go +++ b/types/node_test.go @@ -4,6 +4,7 @@ import ( "context" "encoding/json" "math" + "reflect" "testing" enginemocks "github.com/projecteru2/core/engine/mocks" @@ -202,3 +203,34 @@ func TestAddNodeOptions(t *testing.T) { o.Normalize() assert.EqualValues(t, 3, o.Storage) } + +func TestNodeWithResource(t *testing.T) { + n := Node{ + CPU: CPUMap{"0": 0}, + Volume: VolumeMap{"sda1": 0}, + } + resource := &ResourceMeta{ + CPUQuotaLimit: 0.4, + CPUQuotaRequest: 0.3, + CPU: CPUMap{"0": 30}, + MemoryLimit: 100, + MemoryRequest: 99, + StorageLimit: 88, + StorageRequest: 87, + VolumePlanLimit: MustToVolumePlan(map[string]map[string]int64{"AUTO:/data0:rw:100": {"/sda0": 100}}), + VolumePlanRequest: MustToVolumePlan(map[string]map[string]int64{"AUTO:/data1:rw:101": {"sda1": 101}}), + } + n.RecycleResources(resource) + assert.EqualValues(t, -0.3, n.CPUUsed) + assert.True(t, reflect.DeepEqual(n.CPU, CPUMap{"0": 30})) + assert.EqualValues(t, 99, n.MemCap) + assert.EqualValues(t, 87, n.StorageCap) + assert.EqualValues(t, -101, n.VolumeUsed) + + n.PreserveResources(resource) + assert.EqualValues(t, 0, n.CPUUsed) + assert.True(t, reflect.DeepEqual(n.CPU, CPUMap{"0": 0})) + assert.EqualValues(t, 0, n.MemCap) + assert.EqualValues(t, 0, n.StorageCap) + assert.EqualValues(t, 0, n.VolumeUsed) +} diff --git a/types/options.go b/types/options.go index 880a1a287..11d1267de 100644 --- a/types/options.go +++ b/types/options.go @@ -1,14 +1,8 @@ package types -import ( - "bytes" - "io" - "io/ioutil" - "sync" -) - // DeployOptions is options for deploying type DeployOptions struct { + ResourceOpts ResourceOptions Name string // Name of application Entrypoint *Entrypoint // entrypoint Podname string // Name of pod to deploy @@ -28,60 +22,12 @@ type DeployOptions struct { NodeLabels map[string]string // NodeLabels for filter node DeployStrategy string // Deploy strategy Data map[string]ReaderManager // For additional file data - SoftLimit bool // Soft limit memory NodesLimit int // Limit nodes count ProcessIdent string // ProcessIdent ident this deploy IgnoreHook bool // IgnoreHook ignore hook process AfterCreate []string // AfterCreate support run cmds after create RawArgs []byte // RawArgs for raw args processing Lambda bool // indicate is lambda container or not - RawResourceOptions -} - -// RawResourceOptions . -type RawResourceOptions struct { - CPURequest float64 - CPULimit float64 - CPUBind bool - - MemoryRequest int64 - MemoryLimit int64 - MemorySoft bool - - VolumeRequest VolumeBindings - VolumeLimit VolumeBindings - - StorageRequest int64 - StorageLimit int64 -} - -// ReaderManager return Reader under concurrency -type ReaderManager interface { - GetReader() (io.Reader, error) -} - -type readerManager struct { - mux sync.Mutex - r io.ReadSeeker -} - -func (rm *readerManager) GetReader() (_ io.Reader, err error) { - rm.mux.Lock() - defer rm.mux.Unlock() - buf := &bytes.Buffer{} - if _, err = io.Copy(buf, rm.r); err != nil { - return - } - _, err = rm.r.Seek(0, io.SeekStart) - return buf, err -} - -// NewReaderManager converts Reader to ReadSeeker -func NewReaderManager(r io.Reader) (ReaderManager, error) { - bs, err := ioutil.ReadAll(r) - return &readerManager{ - r: bytes.NewReader(bs), - }, err } // RunAndWaitOptions is options for running and waiting @@ -179,22 +125,9 @@ type ExecuteContainerOptions struct { // ReallocOptions . type ReallocOptions struct { - IDs []string - CPU float64 - Memory int64 - Storage int64 - Volumes VolumeBindings - BindCPUOpt TriOptions - MemoryLimitOpt TriOptions - - CPURequest float64 - CPULimit float64 - MemoryRequest int64 - MemoryLimit int64 - StorageRequest int64 - StorageLimit int64 - VolumeRequest VolumeBindings - VolumeLimit VolumeBindings + ID string + CPUBindOpts TriOptions + ResourceOpts ResourceOptions } // TriOptions . diff --git a/types/resource.go b/types/resource.go index 35d203a82..519a1048a 100644 --- a/types/resource.go +++ b/types/resource.go @@ -1,5 +1,46 @@ package types +import ( + "encoding/json" + "sort" +) + +// ResourceOptions for create/realloc/replace +type ResourceOptions struct { + CPUQuotaRequest float64 + CPUQuotaLimit float64 + CPUBind bool + + MemoryRequest int64 + MemoryLimit int64 + + VolumeRequest VolumeBindings + VolumeLimit VolumeBindings + + StorageRequest int64 + StorageLimit int64 +} + +// ResourceMeta for messages and container to store +type ResourceMeta struct { + CPUQuotaRequest float64 + CPUQuotaLimit float64 + CPU ResourceMap + NUMANode string + + MemoryRequest int64 + MemoryLimit int64 + + VolumeRequest VolumeBindings + VolumeLimit VolumeBindings + VolumePlanRequest VolumePlan + VolumePlanLimit VolumePlan + VolumeChanged bool + + StorageRequest int64 + StorageLimit int64 +} + // ResourceType . type ResourceType int @@ -24,3 +65,172 @@ var ( // AllResourceTypes . AllResourceTypes = [...]ResourceType{ResourceCPU, ResourceMemory, ResourceVolume, ResourceStorage} ) + +// ResourceMap is cpu core map +// ResourceMap {["0"]10000, ["1"]10000} +type ResourceMap map[string]int64 + +// Total show total cpu +// Total quotas +func (c ResourceMap) Total() int64 { + var count int64 + for _, value := range c { + count += value + } + return count +} + +// Add return cpu +func (c ResourceMap) Add(q ResourceMap) { + for label, value := range q { + if _, ok := c[label]; !ok { + c[label] = value + } else { + c[label] += value + } + } +} + +// Sub decrease cpus +func (c ResourceMap) Sub(q ResourceMap) { + for label, value := range q { + if _, ok := c[label]; ok { + c[label] -= value + } + } +} + +// CPUMap is cpu core map +// CPUMap {["0"]10000, ["1"]10000} +type CPUMap = ResourceMap + +// VolumeMap is volume map +// VolumeMap {["/data1"]1073741824, ["/data2"]1048576} +type VolumeMap = ResourceMap + +// GetResourceID returns device name such as "/sda0" +// GetResourceID only works for VolumeMap with single key +func (c VolumeMap) GetResourceID() (key string) { + for k := range c { + key = k + break + } + return +} + +// GetRation returns scheduled size from device +// GetRation only works for VolumeMap with single key +func (c VolumeMap) GetRation() int64 { + return c[c.GetResourceID()] +} + +// SplitByUsed . +func (c VolumeMap) SplitByUsed(init VolumeMap) (VolumeMap, VolumeMap) { + used := VolumeMap{} + unused := VolumeMap{} + for mountDir, freeSpace := range c { + vmap := used + if init[mountDir] == freeSpace { + vmap = unused + } + vmap.Add(VolumeMap{mountDir: freeSpace}) + } + return used, unused +} + +// VolumePlan is map from volume string to volumeMap: {"AUTO:/data:rw:100": VolumeMap{"/sda1": 100}} +type VolumePlan map[VolumeBinding]VolumeMap + +// MakeVolumePlan creates VolumePlan pointer by volume strings and scheduled VolumeMaps +func MakeVolumePlan(vbs VolumeBindings, distribution []VolumeMap) VolumePlan { + sort.Slice(vbs, func(i, j int) bool { return vbs[i].SizeInBytes < vbs[j].SizeInBytes }) + sort.Slice(distribution, func(i, j int) bool { return distribution[i].GetRation() < distribution[j].GetRation() }) + + volumePlan := VolumePlan{} + for idx, vb := range vbs { + if vb.RequireSchedule() { + volumePlan[*vb] = distribution[idx] + } + } + return volumePlan +} + +// UnmarshalJSON . +func (p *VolumePlan) UnmarshalJSON(b []byte) (err error) { + if *p == nil { + *p = VolumePlan{} + } + plan := map[string]VolumeMap{} + if err = json.Unmarshal(b, &plan); err != nil { + return err + } + for volume, vmap := range plan { + vb, err := NewVolumeBinding(volume) + if err != nil { + return err + } + (*p)[*vb] = vmap + } + return +} + +// MarshalJSON . +func (p VolumePlan) MarshalJSON() ([]byte, error) { + plan := map[string]VolumeMap{} + for vb, vmap := range p { + plan[vb.ToString(false)] = vmap + } + return json.Marshal(plan) +} + +// ToLiteral returns literal VolumePlan +func (p VolumePlan) ToLiteral() map[string]map[string]int64 { + plan := map[string]map[string]int64{} + for vb, volumeMap := range p { + plan[vb.ToString(false)] = volumeMap + } + return plan +} + +// IntoVolumeMap Merge return one VolumeMap with all in VolumePlan added +func (p VolumePlan) IntoVolumeMap() VolumeMap { + volumeMap := VolumeMap{} + for _, v := range p { + volumeMap.Add(v) + } + return volumeMap +} + +// GetVolumeMap looks up VolumeMap according to volume destination directory +func (p VolumePlan) GetVolumeMap(vb *VolumeBinding) (volMap VolumeMap, volume VolumeBinding) { + for volume, volMap := range p { + if vb.Destination == volume.Destination { + return volMap, volume + } + } + return +} + +// Compatible return true if new bindings stick to the old bindings +func (p VolumePlan) Compatible(oldPlan VolumePlan) bool { + for volume, oldBinding := range oldPlan { + newBinding, _ := p.GetVolumeMap(&volume) // nolint + // newBinding is ok to be nil when reallocing requires less volumes than before + if newBinding != nil && newBinding.GetResourceID() != oldBinding.GetResourceID() { + // unlimited binding, modify binding source + if newBinding.GetRation() == 0 { + //p[v] = VolumeMap{oldBinding.GetResourceID(): 0} + continue + } + return false + } + } + return true +} + +// Merge . +func (p VolumePlan) Merge(p2 VolumePlan) { + for vb, vm := range p2 { + p[vb] = vm + } +} diff --git a/types/resource_test.go b/types/resource_test.go new file mode 100644 index 000000000..4fd87f0cf --- /dev/null +++ b/types/resource_test.go @@ -0,0 +1,3 @@ +package types + +// TODO move node_test to here diff --git a/types/stream.go b/types/stream.go new file mode 100644 index 000000000..9108d6a0d --- /dev/null +++ b/types/stream.go @@ -0,0 +1,37 @@ +package types + +import ( + "bytes" + "io" + "io/ioutil" + "sync" +) + +// ReaderManager return Reader under concurrency +type ReaderManager interface { + GetReader() (io.Reader, error) +} + +// NewReaderManager converts Reader to ReadSeeker +func NewReaderManager(r io.Reader) (ReaderManager, error) { + bs, err := ioutil.ReadAll(r) + return &readerManager{ + r: bytes.NewReader(bs), + }, err +} + +type readerManager struct { + mux sync.Mutex + r io.ReadSeeker +} + +func (rm *readerManager) GetReader() (_ io.Reader, err error) { + rm.mux.Lock() + defer rm.mux.Unlock() + buf := &bytes.Buffer{} + if _, err = io.Copy(buf, rm.r); err != nil { + return + } + _, err = rm.r.Seek(0, io.SeekStart) + return buf, err +} diff --git a/types/stream_test.go b/types/stream_test.go new file mode 100644 index 000000000..2135d42b6 --- /dev/null +++ b/types/stream_test.go @@ -0,0 +1,19 @@ +package types + +import ( + "io/ioutil" + "strings" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestGetReader(t *testing.T) { + reader := strings.NewReader("aaa") + rm, err := NewReaderManager(reader) + assert.Nil(t, err) + reader2, err := rm.GetReader() + assert.Nil(t, err) + bs, err := ioutil.ReadAll(reader2) + assert.Equal(t, "aaa", string(bs)) +} diff --git a/utils/transaction.go b/utils/transaction.go index 59297e97e..cb0222962 100644 --- a/utils/transaction.go +++ b/utils/transaction.go @@ -11,11 +11,16 @@ import ( type contextFunc = func(context.Context) error // Txn provides unified API to perform txn -func Txn(ctx context.Context, cond contextFunc, then contextFunc, rollback contextFunc, ttl time.Duration) error { - var txnErr error +func Txn(ctx context.Context, cond contextFunc, then contextFunc, rollback func(context.Context, bool) error, ttl time.Duration) (txnErr error) { + var condErr, thenErr error + txnCtx, txnCancel := context.WithTimeout(ctx, ttl) defer txnCancel() defer func() { // rollback + txnErr = condErr + if txnErr == nil { + txnErr = thenErr + } if txnErr == nil { return } @@ -28,13 +33,14 @@ func Txn(ctx context.Context, cond contextFunc, then contextFunc, rollback conte // forbid interrupting rollback rollbackCtx, rollBackCancel := context.WithTimeout(context.Background(), ttl) defer rollBackCancel() - if err := rollback(rollbackCtx); err != nil { + failureByCond := condErr != nil + if err := rollback(rollbackCtx, failureByCond); err != nil { log.Errorf("[txn] txn failed but rollback also failed, %v", err) } }() // let caller decide process then or not - if txnErr = cond(txnCtx); txnErr == nil && then != nil { + if condErr = cond(txnCtx); condErr == nil && then != nil { // no rollback and forbid interrupting further process var thenCtx context.Context = txnCtx var thenCancel context.CancelFunc @@ -42,7 +48,7 @@ func Txn(ctx context.Context, cond contextFunc, then contextFunc, rollback conte thenCtx, thenCancel = context.WithTimeout(context.Background(), ttl) defer thenCancel() } - txnErr = then(thenCtx) + thenErr = then(thenCtx) } return txnErr diff --git a/utils/transaction_test.go b/utils/transaction_test.go index 5e8ae029e..16851f716 100644 --- a/utils/transaction_test.go +++ b/utils/transaction_test.go @@ -17,7 +17,7 @@ func TestTxn(t *testing.T) { return err1 }, nil, - func(context.Context) error { + func(context.Context, bool) error { return errors.New("error 2") }, 10*time.Second,