diff --git a/Makefile b/Makefile index 30fc33116..15f8b8089 100644 --- a/Makefile +++ b/Makefile @@ -22,7 +22,7 @@ deps: env GO111MODULE=on go mod vendor binary: - CGO_ENABLED=0 go build -ldflags "$(GO_LDFLAGS)" -o eru-core + CGO_ENABLED=0 go build -ldflags "$(GO_LDFLAGS)" -gcflags=all=-G=3 -o eru-core build: deps binary @@ -31,7 +31,6 @@ test: deps unit-test mock: deps mockery --dir vendor/google.golang.org/grpc --output 3rdmocks --name ServerStream mockery --dir vendor/github.com/docker/docker/client --output engine/docker/mocks --name APIClient - mockery --dir scheduler --output scheduler/mocks --name Scheduler mockery --dir source --output source/mocks --name Source mockery --dir store --output store/mocks --name Store mockery --dir engine --output engine/mocks --name API @@ -41,6 +40,7 @@ mock: deps mockery --dir store/etcdv3/meta --output store/etcdv3/meta/mocks --all mockery --dir vendor/go.etcd.io/etcd/client/v3 --output store/etcdv3/meta/mocks --name Txn mockery --dir rpc/gen/ --output rpc/mocks --name CoreRPC_RunAndWaitServer + mockery --dir resources --output resources/mocks --name Plugin .ONESHELL: @@ -49,22 +49,21 @@ cloc: unit-test: go vet `go list ./... | grep -v '/vendor/' | grep -v '/tools'` && \ - go test -race -timeout 240s -count=1 -cover ./utils/... \ + go test -race -timeout 240s -count=1 -vet=off -cover ./utils/... \ ./types/... \ ./store/etcdv3/. \ ./store/etcdv3/embedded/. \ ./store/etcdv3/meta/. \ ./source/common/... \ ./strategy/... \ - ./scheduler/complex/... \ ./rpc/. \ ./lock/etcdlock/... \ ./auth/simple/... \ ./discovery/helium... \ - ./resources/types/. \ - ./resources/storage/... \ - ./resources/volume/... \ - ./resources/cpumem/... \ + ./resources/cpumem/models/... \ + ./resources/cpumem/schedule/... \ + ./resources/volume/models/... \ + ./resources/volume/schedule/... \ ./wal/. \ ./wal/kv/. \ ./store/redis/... \ diff --git a/cluster/calcium/build.go b/cluster/calcium/build.go index e6e70bd0d..06bc2adfe 100644 --- a/cluster/calcium/build.go +++ b/cluster/calcium/build.go @@ -75,8 +75,22 @@ func (c *Calcium) selectBuildNode(ctx context.Context) (*types.Node, error) { return nil, errors.WithStack(types.ErrInsufficientNodes) } // get idle max node - node, err := c.scheduler.MaxIdleNode(nodes) - return node, err + return c.getMostIdleNode(ctx, nodes) +} + +func (c *Calcium) getMostIdleNode(ctx context.Context, nodes []*types.Node) (*types.Node, error) { + nodeNames := []string{} + nodeMap := map[string]*types.Node{} + for _, node := range nodes { + nodeNames = append(nodeNames, node.Name) + nodeMap[node.Name] = node + } + + mostIdleNode, err := c.resource.GetMostIdleNode(ctx, nodeNames) + if err != nil { + return nil, err + } + return nodeMap[mostIdleNode], nil } func (c *Calcium) buildFromSCM(ctx context.Context, node *types.Node, opts *types.BuildOptions) ([]string, io.ReadCloser, error) { diff --git a/cluster/calcium/build_test.go b/cluster/calcium/build_test.go index 6bc155c53..e9c610604 100644 --- a/cluster/calcium/build_test.go +++ b/cluster/calcium/build_test.go @@ -8,7 +8,8 @@ import ( "testing" enginemocks "github.com/projecteru2/core/engine/mocks" - schedulermocks "github.com/projecteru2/core/scheduler/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" @@ -82,13 +83,18 @@ func TestBuild(t *testing.T) { Engine: engine, } store.On("GetNodesByPod", mock.AnythingOfType("*context.emptyCtx"), mock.Anything, mock.Anything, mock.Anything).Return([]*types.Node{node}, nil) - scheduler := &schedulermocks.Scheduler{} - c.scheduler = scheduler - // failed by MaxIdleNode - scheduler.On("MaxIdleNode", mock.AnythingOfType("[]*types.Node")).Return(nil, types.ErrBadMeta).Once() + + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + + // failed by plugin error + plugin.On("GetMostIdleNode", mock.Anything, mock.Anything).Return(nil, types.ErrGetMostIdleNodeFailed).Once() ch, err = c.BuildImage(ctx, opts) assert.Error(t, err) - scheduler.On("MaxIdleNode", mock.AnythingOfType("[]*types.Node")).Return(node, nil) + + plugin.On("GetMostIdleNode", mock.Anything, mock.Anything).Return(&resources.GetMostIdleNodeResponse{NodeName: node.Name, Priority: 100}, nil) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + }, nil) // create image c.config.Docker.Hub = "test.com" c.config.Docker.Namespace = "test" diff --git a/cluster/calcium/calcium.go b/cluster/calcium/calcium.go index cfb36601c..7815e401a 100644 --- a/cluster/calcium/calcium.go +++ b/cluster/calcium/calcium.go @@ -9,8 +9,9 @@ import ( "github.com/projecteru2/core/discovery" "github.com/projecteru2/core/discovery/helium" "github.com/projecteru2/core/log" - "github.com/projecteru2/core/scheduler" - complexscheduler "github.com/projecteru2/core/scheduler/complex" + "github.com/projecteru2/core/resources" + "github.com/projecteru2/core/resources/cpumem" + "github.com/projecteru2/core/resources/volume" "github.com/projecteru2/core/source" "github.com/projecteru2/core/source/github" "github.com/projecteru2/core/source/gitlab" @@ -24,7 +25,7 @@ import ( type Calcium struct { config types.Config store store.Store - scheduler scheduler.Scheduler + resource *resources.PluginManager source source.Source watcher discovery.Service wal *WAL @@ -41,13 +42,6 @@ func New(config types.Config, t *testing.T) (*Calcium, error) { return nil, logger.Err(context.TODO(), errors.WithStack(err)) } - // set scheduler - potassium, err := complexscheduler.New(config) - if err != nil { - return nil, logger.Err(context.TODO(), errors.WithStack(err)) - } - scheduler.InitSchedulerV1(potassium) - // set scm var scm source.Source scmtype := strings.ToLower(config.Git.SCMType) @@ -67,10 +61,35 @@ func New(config types.Config, t *testing.T) (*Calcium, error) { // set watcher watcher := helium.New(config.GRPCConfig, store) - cal := &Calcium{store: store, config: config, scheduler: potassium, source: scm, watcher: watcher} + // set resource plugin manager + resource, err := resources.NewPluginManager(config) + if err != nil { + return nil, logger.Err(context.TODO(), errors.WithStack(err)) + } + + // load cpumem plugin + cpumem, err := cpumem.NewPlugin(config) + if err != nil { + log.Errorf(context.TODO(), "[NewPluginManager] new cpumem plugin error: %v", err) + return nil, err + } + resource.AddPlugins(cpumem) + + // load volume plugin + volume, err := volume.NewPlugin(config) + if err != nil { + log.Errorf(context.TODO(), "[NewPluginManager] new volume plugin error: %v", err) + return nil, err + } + resource.AddPlugins(volume) + + cal := &Calcium{store: store, config: config, source: scm, watcher: watcher, resource: resource} + cal.wal, err = newWAL(config, cal) cal.identifier = config.Identifier() + go cal.InitMetrics() + return cal, logger.Err(nil, errors.WithStack(err)) //nolint } diff --git a/cluster/calcium/calcium_test.go b/cluster/calcium/calcium_test.go index f622b568f..c25ee4817 100644 --- a/cluster/calcium/calcium_test.go +++ b/cluster/calcium/calcium_test.go @@ -5,38 +5,21 @@ import ( "io/ioutil" "os" "path/filepath" - "sync" "testing" "time" - schedulermocks "github.com/projecteru2/core/scheduler/mocks" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" sourcemocks "github.com/projecteru2/core/source/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" "github.com/projecteru2/core/wal" walmocks "github.com/projecteru2/core/wal/mocks" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" ) -// DummyLock replace lock for testing -type dummyLock struct { - m sync.Mutex -} - -// Lock for lock -func (d *dummyLock) Lock(ctx context.Context) (context.Context, error) { - d.m.Lock() - return context.Background(), nil -} - -// Unlock for unlock -func (d *dummyLock) Unlock(ctx context.Context) error { - d.m.Unlock() - return nil -} - func NewTestCluster() *Calcium { walDir, err := ioutil.TempDir(os.TempDir(), "core.wal.*") if err != nil { @@ -61,7 +44,6 @@ func NewTestCluster() *Calcium { HAKeepaliveInterval: 16 * time.Second, } c.store = &storemocks.Store{} - c.scheduler = &schedulermocks.Scheduler{} c.source = &sourcemocks.Source{} c.wal = &WAL{WAL: &walmocks.WAL{}} @@ -69,6 +51,14 @@ func NewTestCluster() *Calcium { commit := wal.Commit(func() error { return nil }) mwal.On("Log", mock.Anything, mock.Anything).Return(commit, nil) + plugin := &resourcemocks.Plugin{} + plugin.On("Name").Return("mock-plugin") + plugin.On("ResolveNodeResourceInfoToMetrics", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, context.DeadlineExceeded) + if c.resource, err = resources.NewPluginManager(c.config); err != nil { + panic(err) + } + c.resource.AddPlugins(plugin) + return c } diff --git a/cluster/calcium/capacity.go b/cluster/calcium/capacity.go index d7ef747d3..9e07accdd 100644 --- a/cluster/calcium/capacity.go +++ b/cluster/calcium/capacity.go @@ -5,7 +5,6 @@ import ( "github.com/projecteru2/core/log" "github.com/projecteru2/core/resources" - resourcetypes "github.com/projecteru2/core/resources/types" "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" @@ -22,9 +21,14 @@ func (c *Calcium) CalculateCapacity(ctx context.Context, opts *types.DeployOptio } return msg, c.withNodesPodLocked(ctx, opts.NodeFilter, func(ctx context.Context, nodeMap map[string]*types.Node) error { + nodes := []string{} + for node := range nodeMap { + nodes = append(nodes, node) + } + if opts.DeployStrategy != strategy.Dummy { - if _, msg.NodeCapacities, err = c.doAllocResource(ctx, nodeMap, opts); err != nil { - logger.Errorf(ctx, "[Calcium.CalculateCapacity] doAllocResource failed: %+v", err) + if msg.NodeCapacities, err = c.doGetDeployMap(ctx, nodes, opts); err != nil { + logger.Errorf(ctx, "[Calcium.CalculateCapacity] doGetDeployMap failed: %+v", err) return err } @@ -32,48 +36,19 @@ func (c *Calcium) CalculateCapacity(ctx context.Context, opts *types.DeployOptio msg.Total += capacity } } else { - var infos []strategy.Info - msg.Total, _, infos, err = c.doCalculateCapacity(ctx, nodeMap, opts) + var infos map[string]*resources.NodeCapacityInfo + infos, msg.Total, err = c.resource.GetNodesDeployCapacity(ctx, nodes, opts.ResourceOpts) if err != nil { - logger.Errorf(ctx, "[Calcium.CalculateCapacity] doCalculateCapacity failed: %+v", err) + logger.Errorf(ctx, "[Calcium.CalculateCapacity] failed to get nodes capacity: %+v", err) return err } - for _, info := range infos { - msg.NodeCapacities[info.Nodename] = info.Capacity + if msg.Total <= 0 { + return errors.Wrap(types.ErrInsufficientRes, "no node meets all the resource requirements at the same time") + } + for node, info := range infos { + msg.NodeCapacities[node] = info.Capacity } } return nil }) } - -func (c *Calcium) doCalculateCapacity(ctx context.Context, nodeMap map[string]*types.Node, opts *types.DeployOptions) ( - total int, - plans []resourcetypes.ResourcePlans, - infos []strategy.Info, - err error, -) { - if len(nodeMap) == 0 { - return 0, nil, nil, errors.WithStack(types.ErrInsufficientNodes) - } - - resourceRequests, err := resources.MakeRequests(opts.ResourceOpts) - if err != nil { - return 0, nil, nil, err - } - - // select available nodes - if plans, err = resources.SelectNodesByResourceRequests(ctx, resourceRequests, nodeMap); err != nil { - return 0, nil, nil, err - } - - // deploy strategy - infos = strategy.NewInfos(resourceRequests, nodeMap, plans) - for _, info := range infos { - total += info.Capacity - } - log.Debugf(ctx, "[Calcium.doCalculateCapacity] plans: %+v, total: %v", plans, total) - if total <= 0 { - return 0, nil, nil, errors.Wrap(types.ErrInsufficientRes, "no node meets all the resource requirements at the same time") - } - return -} diff --git a/cluster/calcium/capacity_test.go b/cluster/calcium/capacity_test.go index 4b774a51b..f3179bedf 100644 --- a/cluster/calcium/capacity_test.go +++ b/cluster/calcium/capacity_test.go @@ -2,13 +2,13 @@ package calcium import ( "context" + "errors" "testing" enginemocks "github.com/projecteru2/core/engine/mocks" lockmocks "github.com/projecteru2/core/lock/mocks" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - schedulermocks "github.com/projecteru2/core/scheduler/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" @@ -19,16 +19,15 @@ import ( func TestCalculateCapacity(t *testing.T) { c := NewTestCluster() - scheduler.InitSchedulerV1(c.scheduler) ctx := context.Background() store := c.store.(*storemocks.Store) engine := &enginemocks.API{} + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) // pod1 := &types.Pod{Name: "p1"} node1 := &types.Node{ NodeMeta: types.NodeMeta{ Name: "n1", - CPU: types.CPUMap{"0": 100, "1": 100}, }, Engine: engine, } @@ -37,84 +36,62 @@ func TestCalculateCapacity(t *testing.T) { lock.On("Lock", mock.Anything).Return(context.TODO(), nil) lock.On("Unlock", mock.Anything).Return(nil) store.On("CreateLock", mock.Anything, mock.Anything).Return(lock, nil) - // failed by wrong resource + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + }, nil) + // failed by call plugin opts := &types.DeployOptions{ Entrypoint: &types.Entrypoint{ Name: "entry", }, - ResourceOpts: types.ResourceOptions{ - CPUBind: true, - CPUQuotaRequest: 0, - }, + ResourceOpts: types.WorkloadResourceOpts{}, DeployStrategy: strategy.Auto, NodeFilter: types.NodeFilter{ Includes: []string{"n1"}, }, + Count: 3, } + plugin.On("GetNodesDeployCapacity", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("not implemented")).Once() _, err := c.CalculateCapacity(ctx, opts) assert.Error(t, err) - opts.ResourceOpts.CPUBind = false - opts.ResourceOpts.CPUQuotaRequest = 0.5 - opts.Count = 5 - sched := c.scheduler.(*schedulermocks.Scheduler) - // define scheduleInfos - scheduleInfos := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "n1", - MemCap: 100, + + // failed by get deploy status + plugin.On("GetNodesDeployCapacity", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodesDeployCapacityResponse{ + Nodes: map[string]*resources.NodeCapacityInfo{ + "n1": { + NodeName: "n1", + Capacity: 10, + Usage: 0.5, + Rate: 0.5, + Weight: 100, }, - Capacity: 10, }, - } - sched.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos, 5, nil).Twice() - sched.On("SelectStorageNodes", mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos, 5, nil).Twice() - sched.On("SelectVolumeNodes", mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos, nil, 5, nil).Twice() - store.On("MakeDeployStatus", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Once() - r, err := c.CalculateCapacity(ctx, opts) + Total: 0, + }, nil) + store.On("GetDeployStatus", mock.Anything, mock.Anything, mock.Anything).Return(nil, types.ErrNoETCD).Once() + _, err = c.CalculateCapacity(ctx, opts) + assert.Error(t, err) + + store.On("GetDeployStatus", mock.Anything, mock.Anything, mock.Anything).Return(map[string]int{"n1": 0}, nil) + + // failed by get deploy plan + opts.DeployStrategy = "FAKE" + _, err = c.CalculateCapacity(ctx, opts) + assert.Error(t, err) + + // strategy: auto + opts.DeployStrategy = strategy.Auto + msg, err := c.CalculateCapacity(ctx, opts) assert.NoError(t, err) - assert.Equal(t, r.Total, 5) + assert.Equal(t, msg.NodeCapacities["n1"], 3) + assert.Equal(t, msg.Total, 3) + + // strategy: dummy opts.DeployStrategy = strategy.Dummy - r, err = c.CalculateCapacity(ctx, opts) + msg, err = c.CalculateCapacity(ctx, opts) assert.NoError(t, err) - assert.Equal(t, r.Total, 10) - sched.AssertExpectations(t) - store.AssertExpectations(t) - - // test for total calculation - // fixed on pull/322 - sched.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]resourcetypes.ScheduleInfo{{ - NodeMeta: types.NodeMeta{Name: "n1"}, - Capacity: 1, - }}, 1, nil).Once() - sched.On("SelectStorageNodes", mock.Anything, mock.Anything, mock.Anything).Return([]resourcetypes.ScheduleInfo{{ - NodeMeta: types.NodeMeta{Name: "n2"}, - Capacity: 1, - }}, 1, nil).Once() - sched.On("SelectVolumeNodes", mock.Anything, mock.Anything, mock.Anything).Return([]resourcetypes.ScheduleInfo{{ - NodeMeta: types.NodeMeta{Name: "n3"}, - Capacity: 1, - }}, nil, 1, nil).Once() - r, err = c.CalculateCapacity(ctx, opts) - assert.Error(t, err, "no node meets all the resource requirements at the same time") - sched.AssertExpectations(t) - store.AssertExpectations(t) + assert.Equal(t, msg.NodeCapacities["n1"], 10) + assert.Equal(t, msg.Total, 10) - // continue - sched.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]resourcetypes.ScheduleInfo{{ - NodeMeta: types.NodeMeta{Name: "n1"}, - Capacity: 1, - }}, 1, nil).Once() - sched.On("SelectStorageNodes", mock.Anything, mock.Anything, mock.Anything).Return([]resourcetypes.ScheduleInfo{{ - NodeMeta: types.NodeMeta{Name: "n1"}, - Capacity: 1, - }}, 1, nil).Once() - sched.On("SelectVolumeNodes", mock.Anything, mock.Anything, mock.Anything).Return([]resourcetypes.ScheduleInfo{{ - NodeMeta: types.NodeMeta{Name: "n2"}, - Capacity: 1, - }}, nil, 1, nil).Once() - r, err = c.CalculateCapacity(ctx, opts) - assert.Error(t, err, "no node meets all the resource requirements at the same time") - sched.AssertExpectations(t) store.AssertExpectations(t) } diff --git a/cluster/calcium/create.go b/cluster/calcium/create.go index 5b66488fb..fa58b4d8a 100644 --- a/cluster/calcium/create.go +++ b/cluster/calcium/create.go @@ -2,7 +2,6 @@ package calcium import ( "context" - "encoding/json" "fmt" "sync" "time" @@ -13,7 +12,6 @@ import ( enginetypes "github.com/projecteru2/core/engine/types" "github.com/projecteru2/core/log" "github.com/projecteru2/core/metrics" - resourcetypes "github.com/projecteru2/core/resources/types" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" "github.com/projecteru2/core/wal" @@ -48,9 +46,12 @@ func (c *Calcium) doCreateWorkloads(ctx context.Context, opts *types.DeployOptio // 通过 Processing 状态跟踪达成 18 Oct, 2018 var ( - plans []resourcetypes.ResourcePlans deployMap map[string]int rollbackMap map[string][]int + // map[node][]engineArgs + engineArgsMap = map[string][]types.EngineArgs{} + // map[node][]map[plugin]resourceArgs + resourceArgsMap = map[string][]map[string]types.WorkloadResourceArgs{} ) utils.SentryGo(func() { @@ -98,37 +99,45 @@ func (c *Calcium) doCreateWorkloads(ctx context.Context, opts *types.DeployOptio } }() return c.withNodesPodLocked(ctx, opts.NodeFilter, func(ctx context.Context, nodeMap map[string]*types.Node) (err error) { - // calculate plans - if plans, deployMap, err = c.doAllocResource(ctx, nodeMap, opts); err != nil { + nodeNames := []string{} + nodes := []*types.Node{} + for nodeName, node := range nodeMap { + nodeNames = append(nodeNames, nodeName) + nodes = append(nodes, node) + } + + if resourceCommit, err = c.wal.Log(eventWorkloadResourceAllocated, nodes); err != nil { + return errors.WithStack(err) + } + + deployMap, err = c.doGetDeployMap(ctx, nodeNames, opts) + if err != nil { return err } // commit changes - nodes := []*types.Node{} processingCommits = make(map[string]wal.Commit) - for nodename, deploy := range deployMap { - for _, plan := range plans { - plan.ApplyChangesOnNode(nodeMap[nodename], utils.Range(deploy)...) + for nodeName, deploy := range deployMap { + nodes = append(nodes, nodeMap[nodeName]) + if engineArgsMap[nodeName], resourceArgsMap[nodeName], err = c.resource.Alloc(ctx, nodeName, deploy, opts.ResourceOpts); err != nil { + return errors.WithStack(err) } - nodes = append(nodes, nodeMap[nodename]) - processing := opts.GetProcessing(nodename) - if processingCommits[nodename], err = c.wal.Log(eventProcessingCreated, processing); err != nil { + + processing := opts.GetProcessing(nodeName) + if processingCommits[nodeName], err = c.wal.Log(eventProcessingCreated, processing); err != nil { return errors.WithStack(err) } if err = c.store.CreateProcessing(ctx, processing, deploy); err != nil { return errors.WithStack(err) } } - if resourceCommit, err = c.wal.Log(eventWorkloadResourceAllocated, nodes); err != nil { - return errors.WithStack(err) - } - return errors.WithStack(c.store.UpdateNodes(ctx, nodes...)) + return nil }) }, // then: deploy workloads func(ctx context.Context) (err error) { - rollbackMap, err = c.doDeployWorkloads(ctx, ch, opts, plans, deployMap) + rollbackMap, err = c.doDeployWorkloads(ctx, ch, opts, engineArgsMap, resourceArgsMap, deployMap) return err }, @@ -139,10 +148,10 @@ func (c *Calcium) doCreateWorkloads(ctx context.Context, opts *types.DeployOptio } for nodename, rollbackIndices := range rollbackMap { if e := c.withNodePodLocked(ctx, nodename, func(ctx context.Context, node *types.Node) error { - for _, plan := range plans { - plan.RollbackChangesOnNode(node, rollbackIndices...) // nolint:scopelint - } - return errors.WithStack(c.store.UpdateNodes(ctx, node)) + resourceArgsToRollback := utils.Map(rollbackIndices, func(idx int) map[string]types.WorkloadResourceArgs { + return resourceArgsMap[nodename][idx] + }) + return c.resource.RollbackAlloc(ctx, nodename, resourceArgsToRollback) }); e != nil { err = logger.Err(ctx, e) } @@ -157,7 +166,13 @@ func (c *Calcium) doCreateWorkloads(ctx context.Context, opts *types.DeployOptio return ch } -func (c *Calcium) doDeployWorkloads(ctx context.Context, ch chan *types.CreateWorkloadMessage, opts *types.DeployOptions, plans []resourcetypes.ResourcePlans, deployMap map[string]int) (_ map[string][]int, err error) { +func (c *Calcium) doDeployWorkloads(ctx context.Context, + ch chan *types.CreateWorkloadMessage, + opts *types.DeployOptions, + engineArgsMap map[string][]types.EngineArgs, + resourceArgsMap map[string][]map[string]types.WorkloadResourceArgs, + deployMap map[string]int) (_ map[string][]int, err error) { + wg := sync.WaitGroup{} wg.Add(len(deployMap)) syncRollbackMap := hashmap.HashMap{} @@ -173,7 +188,7 @@ func (c *Calcium) doDeployWorkloads(ctx context.Context, ch chan *types.CreateWo utils.SentryGo(func(nodename string, deploy, seq int) func() { return func() { defer wg.Done() - if indices, err := c.doDeployWorkloadsOnNode(ctx, ch, nodename, opts, deploy, plans, seq); err != nil { + if indices, err := c.doDeployWorkloadsOnNode(ctx, ch, nodename, opts, deploy, engineArgsMap[nodename], resourceArgsMap[nodename], seq); err != nil { syncRollbackMap.Set(nodename, indices) } } @@ -196,8 +211,16 @@ func (c *Calcium) doDeployWorkloads(ctx context.Context, ch chan *types.CreateWo } // deploy scheduled workloads on one node -func (c *Calcium) doDeployWorkloadsOnNode(ctx context.Context, ch chan *types.CreateWorkloadMessage, nodename string, opts *types.DeployOptions, deploy int, plans []resourcetypes.ResourcePlans, seq int) (indices []int, err error) { - logger := log.WithField("Calcium", "doDeployWorkloadsOnNode").WithField("nodename", nodename).WithField("opts", opts).WithField("deploy", deploy).WithField("plans", plans).WithField("seq", seq) +func (c *Calcium) doDeployWorkloadsOnNode(ctx context.Context, + ch chan *types.CreateWorkloadMessage, + nodename string, + opts *types.DeployOptions, + deploy int, + engineArgs []types.EngineArgs, + resourceArgs []map[string]types.WorkloadResourceArgs, + seq int) (indices []int, err error) { + + logger := log.WithField("Calcium", "doDeployWorkloadsOnNode").WithField("nodename", nodename).WithField("opts", opts).WithField("deploy", deploy).WithField("seq", seq) node, err := c.doGetAndPrepareNode(ctx, nodename, opts.Image) if err != nil { for i := 0; i < deploy; i++ { @@ -228,18 +251,12 @@ func (c *Calcium) doDeployWorkloadsOnNode(ctx context.Context, ch chan *types.Cr ch <- createMsg }() - r := &types.ResourceMeta{} - o := resourcetypes.DispenseOptions{ - Node: node, - Index: idx, - } - for _, plan := range plans { - if r, e = plan.Dispense(o, r); e != nil { - return - } + createMsg.EngineArgs = engineArgs[idx] + createMsg.ResourceArgs = map[string]types.WorkloadResourceArgs{} + for k, v := range resourceArgs[idx] { + createMsg.ResourceArgs[k] = v } - createMsg.ResourceMeta = *r createOpts := c.doMakeWorkloadOptions(ctx, seq+idx, createMsg, opts, node) e = c.doDeployOneWorkload(ctx, node, opts, createMsg, createOpts, true) } @@ -275,31 +292,25 @@ func (c *Calcium) doDeployOneWorkload( ) (err error) { logger := log.WithField("Calcium", "doDeployWorkload").WithField("nodename", node.Name).WithField("opts", opts).WithField("msg", msg) workload := &types.Workload{ - 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, - CreateTime: time.Now().Unix(), + ResourceArgs: types.ResourceMeta{}, + EngineArgs: msg.EngineArgs, + 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, + CreateTime: time.Now().Unix(), } + // copy resource args + for k, v := range msg.ResourceArgs { + workload.ResourceArgs[k] = v + } + var commit wal.Commit defer func() { if commit != nil { @@ -427,11 +438,7 @@ func (c *Calcium) doDeployOneWorkload( func (c *Calcium) doMakeWorkloadOptions(ctx context.Context, no int, msg *types.CreateWorkloadMessage, opts *types.DeployOptions, node *types.Node) *enginetypes.VirtualizationCreateOptions { config := &enginetypes.VirtualizationCreateOptions{} // general - config.CPU = msg.CPU - config.Quota = msg.CPUQuotaLimit - config.Memory = msg.MemoryLimit - config.Storage = msg.StorageLimit - config.NUMANode = msg.NUMANode + config.EngineArgs = msg.EngineArgs config.RawArgs = opts.RawArgs config.Lambda = opts.Lambda config.User = opts.User @@ -439,8 +446,6 @@ func (c *Calcium) doMakeWorkloadOptions(ctx context.Context, no int, msg *types. config.Image = opts.Image config.Stdin = opts.OpenStdin config.Hosts = opts.ExtraHosts - config.Volumes = msg.VolumeLimit.ApplyPlan(msg.VolumePlanLimit).ToStringSlice(false, true) - config.VolumePlan = msg.VolumePlanLimit.ToLiteral() config.Debug = opts.Debug config.Networks = opts.Networks @@ -470,12 +475,6 @@ func (c *Calcium) doMakeWorkloadOptions(ctx context.Context, no int, msg *types. env = append(env, fmt.Sprintf("ERU_POD=%s", opts.Podname)) env = append(env, fmt.Sprintf("ERU_NODE_NAME=%s", node.Name)) env = append(env, fmt.Sprintf("ERU_WORKLOAD_SEQ=%d", no)) - env = append(env, fmt.Sprintf("ERU_MEMORY=%d", msg.MemoryLimit)) - env = append(env, fmt.Sprintf("ERU_STORAGE=%d", msg.StorageLimit)) - if msg.CPU != nil { - bs, _ := json.Marshal(msg.CPU) - env = append(env, fmt.Sprintf("ERU_CPU=%s", bs)) - } config.Env = env // basic labels, bind to LabelMeta config.Labels = map[string]string{ diff --git a/cluster/calcium/create_test.go b/cluster/calcium/create_test.go index 799cd8b57..191bdf9d4 100644 --- a/cluster/calcium/create_test.go +++ b/cluster/calcium/create_test.go @@ -5,19 +5,19 @@ import ( "testing" "github.com/pkg/errors" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + enginemocks "github.com/projecteru2/core/engine/mocks" enginetypes "github.com/projecteru2/core/engine/types" lockmocks "github.com/projecteru2/core/lock/mocks" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - schedulermocks "github.com/projecteru2/core/scheduler/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" "github.com/projecteru2/core/wal" walmocks "github.com/projecteru2/core/wal/mocks" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" ) func TestCreateWorkload(t *testing.T) { @@ -62,23 +62,6 @@ func TestCreateWorkload(t *testing.T) { _, err = c.CreateWorkload(ctx, opts) assert.Error(t, err) opts.Entrypoint.Name = "some-nice-entrypoint" - - // failed by memory check - opts.ResourceOpts = types.ResourceOptions{MemoryLimit: -1} - store.On("GetNodesByPod", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, nil) - ch, err := c.CreateWorkload(ctx, opts) - assert.Nil(t, err) - for m := range ch { - assert.Error(t, m.Error) - } - - // failed by CPUQuota - opts.ResourceOpts = types.ResourceOptions{CPUQuotaLimit: -1, MemoryLimit: 1} - ch, err = c.CreateWorkload(ctx, opts) - assert.Nil(t, err) - for m := range ch { - assert.Error(t, m.Error) - } } func TestCreateWorkloadTxn(t *testing.T) { @@ -89,7 +72,7 @@ func TestCreateWorkloadTxn(t *testing.T) { Count: 2, DeployStrategy: strategy.Auto, Podname: "p1", - ResourceOpts: types.ResourceOptions{CPUQuotaLimit: 1}, + ResourceOpts: types.WorkloadResourceOpts{}, Image: "zc:test", Entrypoint: &types.Entrypoint{ Name: "good-entrypoint", @@ -97,10 +80,8 @@ func TestCreateWorkloadTxn(t *testing.T) { } store := &storemocks.Store{} - sche := &schedulermocks.Scheduler{} - scheduler.InitSchedulerV1(sche) c.store = store - c.scheduler = sche + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) node1, node2 := nodes[0], nodes[1] @@ -116,7 +97,7 @@ func TestCreateWorkloadTxn(t *testing.T) { store.On("CreateProcessing", mock.Anything, mock.Anything, mock.Anything).Return(nil) store.On("DeleteProcessing", mock.Anything, mock.Anything, mock.Anything).Return(nil) - // doAllocResource fails: MakeDeployStatus + // doAllocResource fails: GetNodesDeployCapacity lock := &lockmocks.DistributedLock{} lock.On("Lock", mock.Anything).Return(context.Background(), nil) lock.On("Unlock", mock.Anything).Return(nil) @@ -133,72 +114,80 @@ func TestCreateWorkloadTxn(t *testing.T) { } return }, nil) - sche.On("SelectStorageNodes", mock.Anything, mock.AnythingOfType("[]resourcetypes.ScheduleInfo"), mock.AnythingOfType("int64")).Return(func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ int64) []resourcetypes.ScheduleInfo { - return scheduleInfos - }, len(nodes), nil) - sche.On("SelectStorageNodes", mock.Anything, mock.AnythingOfType("[]types.ScheduleInfo"), mock.AnythingOfType("int64")).Return(func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ int64) []resourcetypes.ScheduleInfo { - return scheduleInfos - }, len(nodes), nil) - sche.On("SelectVolumeNodes", mock.Anything, mock.AnythingOfType("[]types.ScheduleInfo"), mock.AnythingOfType("types.VolumeBindings")).Return(func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ types.VolumeBindings) []resourcetypes.ScheduleInfo { - return scheduleInfos - }, nil, len(nodes), nil) - sche.On("SelectMemoryNodes", mock.Anything, mock.AnythingOfType("[]types.ScheduleInfo"), mock.AnythingOfType("float64"), mock.AnythingOfType("int64")).Return( - func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ float64, _ int64) []resourcetypes.ScheduleInfo { - for i := range scheduleInfos { - scheduleInfos[i].Capacity = 1 - } - return scheduleInfos - }, len(nodes), nil) - store.On("MakeDeployStatus", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return( - errors.Wrap(context.DeadlineExceeded, "MakeDeployStatus"), - ).Once() store.On("RemoveWorkload", mock.Anything, mock.Anything).Return(nil) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + }, nil) + plugin.On("GetNodesDeployCapacity", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("GetNodesDeployCapacity")).Once() ch, err := c.CreateWorkload(ctx, opts) assert.Nil(t, err) cnt := 0 for m := range ch { cnt++ - assert.True(t, errors.Is(m.Error, context.DeadlineExceeded)) - assert.Error(t, m.Error, "MakeDeployStatus") + assert.Error(t, m.Error, "GetNodesDeployCapacity") } assert.EqualValues(t, 1, cnt) - assert.False(t, walCommitted) + assert.True(t, walCommitted) + walCommitted = false - // commit resource changes fails: UpdateNodes - store.On("MakeDeployStatus", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) - old := strategy.Plans[strategy.Auto] - strategy.Plans[strategy.Auto] = func(ctx context.Context, sis []strategy.Info, need, total, _ int) (map[string]int, error) { - deployInfos := make(map[string]int) - for _, si := range sis { - deployInfos[si.Nodename] = 1 - } - return deployInfos, nil + // doAllocResource fails: GetDeployStatus + store.On("GetDeployStatus", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.Wrap(context.DeadlineExceeded, "GetDeployStatus")).Once() + plugin.On("GetNodesDeployCapacity", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodesDeployCapacityResponse{ + Nodes: map[string]*resources.NodeCapacityInfo{ + node1.Name: { + NodeName: node1.Name, + Capacity: 10, + Usage: 0.5, + Rate: 0.05, + Weight: 100, + }, + node2.Name: { + NodeName: node2.Name, + Capacity: 10, + Usage: 0.5, + Rate: 0.05, + Weight: 100, + }, + }, + Total: 20, + }, nil) + + ch, err = c.CreateWorkload(ctx, opts) + assert.Nil(t, err) + cnt = 0 + for m := range ch { + cnt++ + assert.ErrorIs(t, m.Error, context.DeadlineExceeded) + assert.Error(t, m.Error, "GetDeployStatus") } - defer func() { - strategy.Plans[strategy.Auto] = old - }() - store.On("UpdateNodes", mock.Anything, mock.Anything, mock.Anything).Return(errors.Wrap(context.DeadlineExceeded, "UpdateNodes1")).Once() + assert.EqualValues(t, 1, cnt) + assert.True(t, walCommitted) walCommitted = false + + // doAllocResource fails: Alloc + store.On("GetDeployStatus", mock.Anything, mock.Anything, mock.Anything).Return(map[string]int{}, nil) + plugin.On("GetDeployArgs", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, context.DeadlineExceeded).Once() ch, err = c.CreateWorkload(ctx, opts) assert.Nil(t, err) cnt = 0 for m := range ch { cnt++ - assert.True(t, errors.Is(m.Error, context.DeadlineExceeded)) - assert.Error(t, m.Error, "UpdateNodes1") + assert.Error(t, m.Error, "DeadlineExceeded") } assert.EqualValues(t, 1, cnt) - node1, node2 = nodes[0], nodes[1] - assert.EqualValues(t, 1, node1.CPUUsed) - assert.EqualValues(t, 1, node2.CPUUsed) - node1.CPUUsed = 0 - node2.CPUUsed = 0 assert.True(t, walCommitted) + walCommitted = false // doCreateWorkloadOnNode fails: doGetAndPrepareNode - store.On("UpdateNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil) - store.On("UpdateNodes", mock.Anything, mock.Anything).Return(nil) + plugin.On("GetDeployArgs", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetDeployArgsResponse{ + EngineArgs: []types.EngineArgs{}, + ResourceArgs: []types.WorkloadResourceArgs{}, + }, nil) + plugin.On("SetNodeResourceUsage", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.SetNodeResourceUsageResponse{ + Before: types.NodeResourceArgs{}, + After: types.NodeResourceArgs{}, + }, nil) store.On("GetNode", mock.AnythingOfType("*context.timerCtx"), mock.AnythingOfType("string"), @@ -214,19 +203,14 @@ func TestCreateWorkloadTxn(t *testing.T) { engine.On("ImageLocalDigests", mock.Anything, mock.Anything).Return(nil, errors.Wrap(context.DeadlineExceeded, "ImageLocalDigest")).Twice() engine.On("ImagePull", mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.Wrap(context.DeadlineExceeded, "ImagePull")).Twice() store.On("DeleteProcessing", mock.Anything, mock.Anything, mock.Anything).Return(nil) - walCommitted = false ch, err = c.CreateWorkload(ctx, opts) assert.Nil(t, err) cnt = 0 for m := range ch { cnt++ - assert.Error(t, m.Error) - assert.True(t, errors.Is(m.Error, context.DeadlineExceeded)) assert.Error(t, m.Error, "ImagePull") } assert.EqualValues(t, 2, cnt) - assert.EqualValues(t, 0, node1.CPUUsed) - assert.EqualValues(t, 0, node2.CPUUsed) assert.True(t, walCommitted) // doDeployOneWorkload fails: VirtualizationCreate @@ -246,8 +230,6 @@ func TestCreateWorkloadTxn(t *testing.T) { assert.Error(t, m.Error, "VirtualizationCreate") } assert.EqualValues(t, 2, cnt) - assert.EqualValues(t, 0, node1.CPUUsed) - assert.EqualValues(t, 0, node2.CPUUsed) assert.True(t, walCommitted) // doCreateAndStartWorkload fails: AddWorkload @@ -266,8 +248,6 @@ func TestCreateWorkloadTxn(t *testing.T) { assert.Error(t, m.Error, "AddWorkload") } assert.EqualValues(t, 2, cnt) - assert.EqualValues(t, 0, node1.CPUUsed) - assert.EqualValues(t, 0, node2.CPUUsed) assert.True(t, walCommitted) // doCreateAndStartWorkload fails: first time AddWorkload failed @@ -292,7 +272,6 @@ func TestCreateWorkloadTxn(t *testing.T) { } assert.EqualValues(t, 2, cnt) assert.EqualValues(t, 1, errCnt) - assert.EqualValues(t, 1, node1.CPUUsed+node2.CPUUsed) assert.True(t, walCommitted) store.AssertExpectations(t) engine.AssertExpectations(t) @@ -302,8 +281,7 @@ func newCreateWorkloadCluster(t *testing.T) (*Calcium, []*types.Node) { c := NewTestCluster() c.wal = &WAL{WAL: &walmocks.WAL{}} c.store = &storemocks.Store{} - c.scheduler = &schedulermocks.Scheduler{} - scheduler.InitSchedulerV1(c.scheduler) + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) engine := &enginemocks.API{} pod1 := &types.Pod{Name: "p1"} @@ -344,27 +322,13 @@ func newCreateWorkloadCluster(t *testing.T) (*Calcium, []*types.Node) { return }, nil) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + }, nil) + mwal := c.wal.WAL.(*walmocks.WAL) commit := wal.Commit(func() error { return nil }) mwal.On("Log", mock.Anything, mock.Anything).Return(commit, nil) - sche := c.scheduler.(*schedulermocks.Scheduler) - sche.On("SelectStorageNodes", mock.AnythingOfType("*context.emptyCtx"), mock.AnythingOfType("[]resourcetypes.ScheduleInfo"), mock.AnythingOfType("int64")).Return(func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ int64) []resourcetypes.ScheduleInfo { - return scheduleInfos - }, len(nodes), nil) - sche.On("SelectStorageNodes", mock.Anything, mock.AnythingOfType("[]types.ScheduleInfo"), mock.AnythingOfType("int64")).Return(func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ int64) []resourcetypes.ScheduleInfo { - return scheduleInfos - }, len(nodes), nil) - sche.On("SelectVolumeNodes", mock.Anything, mock.AnythingOfType("[]types.ScheduleInfo"), mock.AnythingOfType("types.VolumeBindings")).Return(func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ types.VolumeBindings) []resourcetypes.ScheduleInfo { - return scheduleInfos - }, nil, len(nodes), nil) - sche.On("SelectMemoryNodes", mock.Anything, mock.AnythingOfType("[]types.ScheduleInfo"), mock.AnythingOfType("float64"), mock.AnythingOfType("int64")).Return( - func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ float64, _ int64) []resourcetypes.ScheduleInfo { - for i := range scheduleInfos { - scheduleInfos[i].Capacity = 1 - } - return scheduleInfos - }, len(nodes), nil) - return c, nodes } diff --git a/cluster/calcium/dissociate.go b/cluster/calcium/dissociate.go index 4e3abef07..4c0f288ee 100644 --- a/cluster/calcium/dissociate.go +++ b/cluster/calcium/dissociate.go @@ -4,6 +4,7 @@ import ( "context" "github.com/projecteru2/core/log" + "github.com/projecteru2/core/resources" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" @@ -33,9 +34,11 @@ func (c *Calcium) DissociateWorkload(ctx context.Context, ids []string) (chan *t ctx, // if func(ctx context.Context) (err error) { - if err = c.store.UpdateNodeResource(ctx, node, &workload.ResourceMeta, types.ActionIncr); err == nil { - log.Infof(ctx, "[DissociateWorkload] Workload %s dissociated", workload.ID) + resourceArgs := map[string]types.WorkloadResourceArgs{} + for plugin, args := range workload.ResourceArgs { + resourceArgs[plugin] = args } + _, _, err = c.resource.SetNodeResourceUsage(ctx, node.Name, nil, nil, []map[string]types.WorkloadResourceArgs{resourceArgs}, true, resources.Decr) return errors.WithStack(err) }, // then @@ -47,7 +50,12 @@ func (c *Calcium) DissociateWorkload(ctx context.Context, ids []string) (chan *t if failedByCond { return nil } - return errors.WithStack(c.store.UpdateNodeResource(ctx, node, &workload.ResourceMeta, types.ActionDecr)) + resourceArgs := map[string]types.WorkloadResourceArgs{} + for plugin, args := range workload.ResourceArgs { + resourceArgs[plugin] = args + } + _, _, err = c.resource.SetNodeResourceUsage(ctx, node.Name, nil, nil, []map[string]types.WorkloadResourceArgs{resourceArgs}, true, resources.Incr) + return errors.WithStack(err) }, c.config.GlobalTimeout, ) diff --git a/cluster/calcium/dissociate_test.go b/cluster/calcium/dissociate_test.go index 7742f61b5..2d42c1c35 100644 --- a/cluster/calcium/dissociate_test.go +++ b/cluster/calcium/dissociate_test.go @@ -6,10 +6,11 @@ import ( "time" lockmocks "github.com/projecteru2/core/lock/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" - "github.com/docker/go-units" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" ) @@ -19,34 +20,30 @@ func TestDissociateWorkload(t *testing.T) { ctx := context.Background() store := &storemocks.Store{} c.store = store + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) lock := &lockmocks.DistributedLock{} lock.On("Lock", mock.Anything).Return(context.TODO(), nil) lock.On("Unlock", mock.Anything).Return(nil) c1 := &types.Workload{ - 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", + ResourceArgs: types.ResourceMeta{}, + ID: "c1", + Podname: "p1", + Nodename: "node1", } node1 := &types.Node{ NodeMeta: types.NodeMeta{ Name: "node1", - MemCap: units.GiB, - CPU: types.CPUMap{"0": 10, "1": 70, "2": 10, "3": 100}, Endpoint: "http://1.1.1.1:1", }, } store.On("GetWorkloads", mock.Anything, mock.Anything).Return([]*types.Workload{c1}, nil) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + }, nil) // failed by lock store.On("GetNode", mock.Anything, "node1").Return(node1, nil) store.On("CreateLock", mock.Anything, mock.Anything).Return(nil, types.ErrNoETCD).Once() @@ -59,7 +56,10 @@ func TestDissociateWorkload(t *testing.T) { store.On("CreateLock", mock.Anything, mock.Anything).Return(lock, nil) // failed by RemoveWorkload - store.On("UpdateNodeResource", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) + plugin.On("SetNodeResourceUsage", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.SetNodeResourceUsageResponse{ + Before: types.NodeResourceArgs{}, + After: types.NodeResourceArgs{}, + }, nil) store.On("RemoveWorkload", mock.Anything, mock.Anything).Return(types.ErrNoETCD).Once() store.On("ListNodeWorkloads", mock.Anything, mock.Anything, mock.Anything).Return(nil, types.ErrNoETCD) ch, err = c.DissociateWorkload(ctx, []string{"c1"}) diff --git a/cluster/calcium/lambda_test.go b/cluster/calcium/lambda_test.go index 97467d43f..aa5b87312 100644 --- a/cluster/calcium/lambda_test.go +++ b/cluster/calcium/lambda_test.go @@ -11,9 +11,8 @@ import ( enginemocks "github.com/projecteru2/core/engine/mocks" enginetypes "github.com/projecteru2/core/engine/types" lockmocks "github.com/projecteru2/core/lock/mocks" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - schedulermocks "github.com/projecteru2/core/scheduler/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" @@ -27,6 +26,7 @@ func TestRunAndWaitFailedThenWALCommitted(t *testing.T) { assert := assert.New(t) c, _ := newCreateWorkloadCluster(t) c.wal = &WAL{WAL: &walmocks.WAL{}} + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) mwal := c.wal.WAL.(*walmocks.WAL) defer mwal.AssertNotCalled(t, "Log") @@ -37,15 +37,14 @@ func TestRunAndWaitFailedThenWALCommitted(t *testing.T) { Count: 2, DeployStrategy: strategy.Auto, Podname: "p1", - ResourceOpts: types.ResourceOptions{CPUQuotaLimit: 1}, + ResourceOpts: types.WorkloadResourceOpts{}, Image: "zc:test", Entrypoint: &types.Entrypoint{ Name: "good-entrypoint", }, } - mstore := c.store.(*storemocks.Store) - mstore.On("MakeDeployStatus", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(fmt.Errorf("err")).Once() + plugin.On("GetNodesDeployCapacity", mock.Anything, mock.Anything, mock.Anything).Return(nil, context.DeadlineExceeded) _, ch, err := c.RunAndWait(context.Background(), opts, make(chan []byte)) assert.NoError(err) @@ -76,7 +75,7 @@ func TestLambdaWithWorkloadIDReturned(t *testing.T) { Count: 2, DeployStrategy: strategy.Auto, Podname: "p1", - ResourceOpts: types.ResourceOptions{CPUQuotaLimit: 1}, + ResourceOpts: types.WorkloadResourceOpts{}, Image: "zc:test", Entrypoint: &types.Entrypoint{ Name: "good-entrypoint", @@ -127,7 +126,7 @@ func TestLambdaWithError(t *testing.T) { Count: 2, DeployStrategy: strategy.Auto, Podname: "p1", - ResourceOpts: types.ResourceOptions{CPUQuotaLimit: 1}, + ResourceOpts: types.WorkloadResourceOpts{}, Image: "zc:test", Entrypoint: &types.Entrypoint{ Name: "good-entrypoint", @@ -189,13 +188,42 @@ func newLambdaCluster(t *testing.T) (*Calcium, []*types.Node) { c, nodes := newCreateWorkloadCluster(t) store := &storemocks.Store{} - sche := &schedulermocks.Scheduler{} - scheduler.InitSchedulerV1(sche) c.store = store - c.scheduler = sche + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) node1, node2 := nodes[0], nodes[1] + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + }, nil) + plugin.On("GetDeployArgs", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetDeployArgsResponse{ + EngineArgs: []types.EngineArgs{}, + ResourceArgs: []types.WorkloadResourceArgs{}, + }, nil) + plugin.On("SetNodeResourceUsage", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.SetNodeResourceUsageResponse{ + Before: types.NodeResourceArgs{}, + After: types.NodeResourceArgs{}, + }, nil) + plugin.On("GetNodesDeployCapacity", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodesDeployCapacityResponse{ + Nodes: map[string]*resources.NodeCapacityInfo{ + node1.Name: { + NodeName: node1.Name, + Capacity: 10, + Usage: 0.5, + Rate: 0.05, + Weight: 100, + }, + node2.Name: { + NodeName: node2.Name, + Capacity: 10, + Usage: 0.5, + Rate: 0.05, + Weight: 100, + }, + }, + Total: 20, + }, nil) + store.On("CreateProcessing", mock.Anything, mock.Anything, mock.Anything).Return(nil) store.On("UpdateProcessing", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) store.On("DeleteProcessing", mock.Anything, mock.Anything, mock.Anything).Return(nil) @@ -216,24 +244,8 @@ func newLambdaCluster(t *testing.T) (*Calcium, []*types.Node) { } return }, nil) - sche.On("SelectStorageNodes", mock.Anything, mock.AnythingOfType("[]resourcetypes.ScheduleInfo"), mock.AnythingOfType("int64")).Return(func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ int64) []resourcetypes.ScheduleInfo { - return scheduleInfos - }, len(nodes), nil) - sche.On("SelectStorageNodes", mock.Anything, mock.AnythingOfType("[]types.ScheduleInfo"), mock.AnythingOfType("int64")).Return(func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ int64) []resourcetypes.ScheduleInfo { - return scheduleInfos - }, len(nodes), nil) - sche.On("SelectVolumeNodes", mock.Anything, mock.AnythingOfType("[]types.ScheduleInfo"), mock.AnythingOfType("types.VolumeBindings")).Return(func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ types.VolumeBindings) []resourcetypes.ScheduleInfo { - return scheduleInfos - }, nil, len(nodes), nil) - sche.On("SelectMemoryNodes", mock.Anything, mock.AnythingOfType("[]types.ScheduleInfo"), mock.AnythingOfType("float64"), mock.AnythingOfType("int64")).Return( - func(_ context.Context, scheduleInfos []resourcetypes.ScheduleInfo, _ float64, _ int64) []resourcetypes.ScheduleInfo { - for i := range scheduleInfos { - scheduleInfos[i].Capacity = 1 - } - return scheduleInfos - }, len(nodes), nil) - store.On("MakeDeployStatus", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) + store.On("GetDeployStatus", mock.Anything, mock.Anything, mock.Anything).Return(map[string]int{}, nil) old := strategy.Plans[strategy.Auto] strategy.Plans[strategy.Auto] = func(ctx context.Context, sis []strategy.Info, need, total, _ int) (map[string]int, error) { deployInfos := make(map[string]int) @@ -246,8 +258,8 @@ func newLambdaCluster(t *testing.T) (*Calcium, []*types.Node) { strategy.Plans[strategy.Auto] = old }() - store.On("UpdateNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil) - store.On("UpdateNodes", mock.Anything, mock.Anything).Return(nil) + //store.On("UpdateNodes", mock.Anything, mock.Anything, mock.Anything).Return(nil) + //store.On("UpdateNodes", mock.Anything, mock.Anything).Return(nil) store.On("GetNode", mock.AnythingOfType("*context.timerCtx"), mock.AnythingOfType("string"), diff --git a/cluster/calcium/lock_test.go b/cluster/calcium/lock_test.go index 82afc3221..eb8a18766 100644 --- a/cluster/calcium/lock_test.go +++ b/cluster/calcium/lock_test.go @@ -7,6 +7,8 @@ import ( enginemocks "github.com/projecteru2/core/engine/mocks" "github.com/projecteru2/core/lock" lockmocks "github.com/projecteru2/core/lock/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" @@ -121,6 +123,11 @@ func TestWithNodesPodLocked(t *testing.T) { ctx := context.Background() store := &storemocks.Store{} c.store = store + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) node1 := &types.Node{ NodeMeta: types.NodeMeta{ @@ -178,6 +185,11 @@ func TestWithNodePodLocked(t *testing.T) { ctx := context.Background() store := &storemocks.Store{} c.store = store + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) node1 := &types.Node{ NodeMeta: types.NodeMeta{ @@ -212,6 +224,11 @@ func TestWithNodesOperationLocked(t *testing.T) { ctx := context.Background() store := &storemocks.Store{} c.store = store + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) node1 := &types.Node{ NodeMeta: types.NodeMeta{ @@ -268,6 +285,11 @@ func TestWithNodeOperationLocked(t *testing.T) { ctx := context.Background() store := &storemocks.Store{} c.store = store + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) node1 := &types.Node{ NodeMeta: types.NodeMeta{ diff --git a/cluster/calcium/metrics.go b/cluster/calcium/metrics.go new file mode 100644 index 000000000..3609117ad --- /dev/null +++ b/cluster/calcium/metrics.go @@ -0,0 +1,46 @@ +package calcium + +import ( + "context" + + "github.com/sanity-io/litter" + + "github.com/projecteru2/core/log" + "github.com/projecteru2/core/metrics" + "github.com/projecteru2/core/utils" +) + +// InitMetrics . +func (c *Calcium) InitMetrics() { + ctx, cancel := context.WithCancel(context.TODO()) + defer cancel() + + metricsDescriptions, err := c.resource.GetMetricsDescription(ctx) + if err != nil { + log.Errorf(ctx, "[InitMetrics] failed to get metrics description, err: %v", err) + return + } + if err = metrics.InitMetrics(c.config, metricsDescriptions); err != nil { + log.Errorf(ctx, "[InitMetrics] failed to init metrics, err: %v", err) + return + } + log.Infof(ctx, "[InitMetrics] init metrics %v success", litter.Sdump(metricsDescriptions)) +} + +// SendNodeMetrics . +func (c *Calcium) SendNodeMetrics(ctx context.Context, nodeName string) { + ctx, cancel := context.WithTimeout(utils.InheritTracingInfo(ctx, context.TODO()), c.config.GlobalTimeout) + defer cancel() + node, err := c.GetNode(ctx, nodeName) + if err != nil { + log.Errorf(ctx, "[SendNodeMetrics] get node %s failed, %v", nodeName, err) + return + } + + nodeMetrics, err := c.resource.ResolveNodeResourceInfoToMetrics(ctx, node.Podname, node.Name, node.ResourceCapacity, node.ResourceUsage) + if err != nil { + log.Errorf(ctx, "[SendNodeMetrics] resolve node %s resource info to metrics failed, %v", nodeName, err) + return + } + metrics.Client.SendMetrics(nodeMetrics...) +} diff --git a/cluster/calcium/node.go b/cluster/calcium/node.go index 3dd096a9c..51eee999e 100644 --- a/cluster/calcium/node.go +++ b/cluster/calcium/node.go @@ -4,7 +4,9 @@ import ( "context" "sort" + enginefactory "github.com/projecteru2/core/engine/factory" "github.com/projecteru2/core/log" + "github.com/projecteru2/core/resources" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" @@ -19,9 +21,49 @@ func (c *Calcium) AddNode(ctx context.Context, opts *types.AddNodeOptions) (*typ if err := opts.Validate(); err != nil { return nil, logger.Err(ctx, err) } - opts.Normalize() - node, err := c.store.AddNode(ctx, opts) - return node, logger.Err(ctx, errors.WithStack(err)) + var resourceCapacity map[string]types.NodeResourceArgs + var resourceUsage map[string]types.NodeResourceArgs + var node *types.Node + var err error + + // check if the node is alive + client, err := enginefactory.GetEngine(ctx, c.config, opts.Nodename, opts.Endpoint, opts.Ca, opts.Cert, opts.Key) + if err != nil { + return nil, errors.WithStack(err) + } + // get node info + nodeInfo, err := client.Info(ctx) + if err != nil { + return nil, errors.WithStack(err) + } + + return node, logger.Err(ctx, utils.Txn( + ctx, + // if: add node resource with resource plugins + func(ctx context.Context) error { + resourceCapacity, resourceUsage, err = c.resource.AddNode(ctx, opts.Nodename, opts.ResourceOpts, nodeInfo) + return errors.WithStack(err) + }, + // then: add node meta in store + func(ctx context.Context) error { + node, err = c.store.AddNode(ctx, opts) + if err != nil { + return errors.WithStack(err) + } + node.ResourceCapacity = resourceCapacity + node.ResourceUsage = resourceUsage + go c.SendNodeMetrics(ctx, node.Name) + return nil + }, + // rollback: remove node with resource plugins + func(ctx context.Context, failureByCond bool) error { + if failureByCond { + return nil + } + return errors.WithStack(c.resource.RemoveNode(ctx, opts.Nodename)) + }, + c.config.GlobalTimeout), + ) } // RemoveNode remove a node @@ -38,7 +80,22 @@ func (c *Calcium) RemoveNode(ctx context.Context, nodename string) error { if len(ws) > 0 { return logger.Err(ctx, errors.WithStack(types.ErrNodeNotEmpty)) } - return logger.Err(ctx, errors.WithStack(c.store.RemoveNode(ctx, node))) + + return logger.Err(ctx, utils.Txn(ctx, + // if: remove node metadata + func(ctx context.Context) error { + return errors.WithStack(c.store.RemoveNode(ctx, node)) + }, + // then: remove node resource metadata + func(ctx context.Context) error { + return errors.WithStack(c.resource.RemoveNode(ctx, nodename)) + }, + // rollback: do nothing + func(ctx context.Context, failureByCond bool) error { + return nil + }, + c.config.GlobalTimeout, + )) }) } @@ -67,6 +124,9 @@ func (c *Calcium) ListPodNodes(ctx context.Context, opts *types.ListNodesOptions if err != nil { logger.Errorf(ctx, "failed to get node %v info: %+v", node.Name, err) } + if err := c.getNodeResourceInfo(ctx, node); err != nil { + logger.Errorf(ctx, "failed to get node %v resource info: %+v", node.Name, err) + } ch <- node } }(node)) @@ -77,13 +137,18 @@ func (c *Calcium) ListPodNodes(ctx context.Context, opts *types.ListNodesOptions } // GetNode get node -func (c *Calcium) GetNode(ctx context.Context, nodename string) (*types.Node, error) { +func (c *Calcium) GetNode(ctx context.Context, nodename string) (node *types.Node, err error) { logger := log.WithField("Calcium", "GetNode").WithField("nodename", nodename) if nodename == "" { return nil, logger.Err(ctx, errors.WithStack(types.ErrEmptyNodeName)) } - node, err := c.store.GetNode(ctx, nodename) - return node, logger.Err(ctx, errors.WithStack(err)) + if node, err = c.store.GetNode(ctx, nodename); err != nil { + return nil, logger.Err(ctx, errors.WithStack(err)) + } + if err = c.getNodeResourceInfo(ctx, node); err != nil { + return nil, logger.Err(ctx, errors.WithStack(err)) + } + return node, nil } // GetNodeEngine get node engine @@ -109,7 +174,6 @@ func (c *Calcium) SetNode(ctx context.Context, opts *types.SetNodeOptions) (*typ var n *types.Node return n, c.withNodePodLocked(ctx, opts.Nodename, func(ctx context.Context, node *types.Node) error { logger.Infof(ctx, "set node") - opts.Normalize(node) n = node n.Bypass = (opts.BypassOpt == types.TriTrue) || (opts.BypassOpt == types.TriKeep && n.Bypass) @@ -119,6 +183,7 @@ func (c *Calcium) SetNode(ctx context.Context, opts *types.SetNodeOptions) (*typ if opts.WorkloadsDown { c.setAllWorkloadsOnNodeDown(ctx, opts.Nodename) } + // update node endpoint if opts.Endpoint != "" { n.Endpoint = opts.Endpoint @@ -131,78 +196,52 @@ func (c *Calcium) SetNode(ctx context.Context, opts *types.SetNodeOptions) (*typ if len(opts.Labels) != 0 { n.Labels = opts.Labels } - // update numa - if len(opts.NUMA) != 0 { - n.NUMA = opts.NUMA - } - // update numa memory - for numaNode, memoryDelta := range opts.DeltaNUMAMemory { - if _, ok := n.NUMAMemory[numaNode]; ok { - n.NUMAMemory[numaNode] += memoryDelta - n.InitNUMAMemory[numaNode] += memoryDelta - if n.NUMAMemory[numaNode] < 0 { - return logger.Err(ctx, errors.WithStack(types.ErrBadMemory)) + + var originNodeResourceCapacity map[string]types.NodeResourceArgs + var err error + + return logger.Err(ctx, utils.Txn(ctx, + // if: update node resource capacity success + func(ctx context.Context) error { + if len(opts.ResourceOpts) == 0 { + return nil } - } - } - if opts.DeltaStorage != 0 { - // update storage - n.StorageCap += opts.DeltaStorage - n.InitStorageCap += opts.DeltaStorage - if n.StorageCap < 0 { - return logger.Err(ctx, errors.WithStack(types.ErrBadStorage)) - } - } - if opts.DeltaMemory != 0 { - // update memory - n.MemCap += opts.DeltaMemory - n.InitMemCap += opts.DeltaMemory - if n.MemCap < 0 { - return logger.Err(ctx, errors.WithStack(types.ErrBadStorage)) - } - } - // update cpu - for cpuID, cpuShare := range opts.DeltaCPU { - _, ok := n.CPU[cpuID] - switch { - case !ok && cpuShare > 0: // incr CPU - n.CPU[cpuID] = cpuShare - n.InitCPU[cpuID] = cpuShare - case ok: // decr share - n.CPU[cpuID] += cpuShare - n.InitCPU[cpuID] += cpuShare - if n.CPU[cpuID] < 0 { - return logger.Err(ctx, errors.WithStack(types.ErrBadCPU)) + + originNodeResourceCapacity, _, err = c.resource.SetNodeResourceCapacity(ctx, n.Name, opts.ResourceOpts, nil, opts.Delta, resources.Incr) + return errors.WithStack(err) + }, + // then: update node metadata + func(ctx context.Context) error { + if err := errors.WithStack(c.store.UpdateNodes(ctx, n)); err != nil { + return err } - if n.InitCPU[cpuID] == 0 { - // decr CPU - delete(n.CPU, cpuID) - delete(n.InitCPU, cpuID) + go c.SendNodeMetrics(ctx, node.Name) + return nil + }, + // rollback: update node resource capacity in reverse + func(ctx context.Context, failureByCond bool) error { + if failureByCond { + return nil } - } - } - // update volume - for volumeDir, changeCap := range opts.DeltaVolume { - _, ok := n.Volume[volumeDir] - switch { - case !ok && changeCap > 0: - n.Volume[volumeDir] = changeCap - n.InitVolume[volumeDir] = changeCap - case ok && changeCap == 0: - delete(n.Volume, volumeDir) - delete(n.InitVolume, volumeDir) - case ok: - n.Volume[volumeDir] += changeCap - n.InitVolume[volumeDir] += changeCap - if n.Volume[volumeDir] < 0 { - return logger.Err(ctx, errors.WithStack(types.ErrBadVolume)) + if len(opts.ResourceOpts) == 0 { + return nil } - } - } - return logger.Err(ctx, errors.WithStack(c.store.UpdateNodes(ctx, n))) + _, _, err = c.resource.SetNodeResourceCapacity(ctx, n.Name, nil, originNodeResourceCapacity, false, resources.Decr) + return errors.WithStack(err) + }, + c.config.GlobalTimeout, + )) }) } +func (c *Calcium) getNodeResourceInfo(ctx context.Context, node *types.Node) (err error) { + if node.ResourceCapacity, node.ResourceUsage, _, err = c.resource.GetNodeResourceInfo(ctx, node.Name, nil, false); err != nil { + log.Errorf(ctx, "[getNodeResourceInfo] failed to get node resource info for node %v, err: %v", node.Name, err) + return errors.WithStack(err) + } + return nil +} + func (c *Calcium) setAllWorkloadsOnNodeDown(ctx context.Context, nodename string) { workloads, err := c.store.ListNodeWorkloads(ctx, nodename, nil) if err != nil { diff --git a/cluster/calcium/node_test.go b/cluster/calcium/node_test.go index 23637e7bd..872cb1773 100644 --- a/cluster/calcium/node_test.go +++ b/cluster/calcium/node_test.go @@ -4,9 +4,12 @@ import ( "context" "testing" + "github.com/projecteru2/core/engine/factory" enginemocks "github.com/projecteru2/core/engine/mocks" enginetypes "github.com/projecteru2/core/engine/types" lockmocks "github.com/projecteru2/core/lock/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" @@ -18,9 +21,15 @@ import ( func TestAddNode(t *testing.T) { c := NewTestCluster() ctx := context.Background() + factory.InitEngineCache(ctx, c.config) + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + name := "test" node := &types.Node{ - NodeMeta: types.NodeMeta{Name: name}, + NodeMeta: types.NodeMeta{ + Name: name, + Endpoint: "endpoint", + }, } store := &storemocks.Store{} @@ -31,6 +40,16 @@ func TestAddNode(t *testing.T) { mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(node, nil) c.store = store + plugin.On("AddNode", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.AddNodeResponse{}, nil) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{ + Capacity: types.NodeResourceArgs{}, + Usage: types.NodeResourceArgs{}, + }, + }, nil) + store.On("GetNode", + mock.Anything, + mock.Anything).Return(node, nil) // fail by validating _, err := c.AddNode(ctx, &types.AddNodeOptions{}) @@ -39,7 +58,7 @@ func TestAddNode(t *testing.T) { n, err := c.AddNode(ctx, &types.AddNodeOptions{ Nodename: "nodename", Podname: "podname", - Endpoint: "endpoint", + Endpoint: "mock://" + name, }) assert.NoError(t, err) assert.Equal(t, n.Name, name) @@ -48,9 +67,7 @@ func TestAddNode(t *testing.T) { func TestRemoveNode(t *testing.T) { c := NewTestCluster() ctx := context.Background() - - // fail by validating - assert.Error(t, c.RemoveNode(ctx, "")) + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) name := "test" node := &types.Node{NodeMeta: types.NodeMeta{Name: name}} @@ -61,6 +78,13 @@ func TestRemoveNode(t *testing.T) { lock.On("Lock", mock.Anything).Return(context.TODO(), nil) lock.On("Unlock", mock.Anything).Return(nil) store.On("CreateLock", mock.Anything, mock.Anything).Return(lock, nil) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{ + Capacity: types.NodeResourceArgs{}, + Usage: types.NodeResourceArgs{}, + }, + }, nil) + plugin.On("RemoveNode", mock.Anything, mock.Anything).Return(&resources.RemoveNodeResponse{}, nil) store.On("GetNode", mock.Anything, @@ -118,6 +142,14 @@ func TestListPodNodes(t *testing.T) { func TestGetNode(t *testing.T) { c := NewTestCluster() ctx := context.Background() + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{ + Capacity: types.NodeResourceArgs{}, + Usage: types.NodeResourceArgs{}, + }, + }, nil) // fail by validating _, err := c.GetNode(ctx, "") @@ -165,9 +197,21 @@ func TestGetNodeEngine(t *testing.T) { func TestSetNode(t *testing.T) { c := NewTestCluster() ctx := context.Background() + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{ + Capacity: types.NodeResourceArgs{}, + Usage: types.NodeResourceArgs{}, + }, + }, nil) + plugin.On("SetNodeResourceUsage", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.SetNodeResourceUsageResponse{ + Before: types.NodeResourceArgs{}, + After: types.NodeResourceArgs{}, + }, nil) + name := "test" node := &types.Node{NodeMeta: types.NodeMeta{Name: name}} - node.Init() store := &storemocks.Store{} c.store = store @@ -236,88 +280,19 @@ func TestSetNode(t *testing.T) { setOpts.Key = "hh" n, err = c.SetNode(ctx, setOpts) assert.NoError(t, err) - // set numa - setOpts.NUMA = types.NUMA{"100": "node1"} - n, err = c.SetNode(ctx, setOpts) - assert.NoError(t, err) - assert.Equal(t, n.NUMA["100"], "node1") - // failed by numa node memory < 0 - n.NUMAMemory = types.NUMAMemory{"node1": 1} - n.InitNUMAMemory = types.NUMAMemory{"node1": 2} - setOpts.DeltaNUMAMemory = types.NUMAMemory{"node1": -10} - n, err = c.SetNode(ctx, setOpts) - assert.Error(t, err) - // succ set numa node memory - n.NUMAMemory = types.NUMAMemory{"node1": 1} - n.InitNUMAMemory = types.NUMAMemory{"node1": 2} - setOpts.DeltaNUMAMemory = types.NUMAMemory{"node1": -1} - n, err = c.SetNode(ctx, setOpts) - assert.NoError(t, err) - assert.Equal(t, n.NUMAMemory["node1"], int64(0)) - setOpts.DeltaNUMAMemory = types.NUMAMemory{} - // failed set storage - n.StorageCap = 1 - n.InitStorageCap = 2 - setOpts.DeltaStorage = -10 - n, err = c.SetNode(ctx, setOpts) - assert.Error(t, err) - // succ set storage - n.StorageCap = 1 - n.InitStorageCap = 2 - setOpts.DeltaStorage = -1 - n, err = c.SetNode(ctx, setOpts) - assert.NoError(t, err) - assert.Equal(t, n.StorageCap, int64(0)) - setOpts.DeltaStorage = 0 - // failed set memory - n.MemCap = 1 - n.InitMemCap = 2 - setOpts.DeltaMemory = -10 - n, err = c.SetNode(ctx, setOpts) - assert.Error(t, err) - // succ set storage - n.MemCap = 1 - n.InitMemCap = 2 - setOpts.DeltaMemory = -1 - n, err = c.SetNode(ctx, setOpts) - assert.NoError(t, err) - assert.Equal(t, n.MemCap, int64(0)) - setOpts.DeltaMemory = 0 - // failed by set cpu - n.CPU = types.CPUMap{"1": 1} - n.InitCPU = types.CPUMap{"1": 2} - setOpts.DeltaCPU = types.CPUMap{"1": -10} - n, err = c.SetNode(ctx, setOpts) - assert.Error(t, err) - // succ set cpu, add and del - n.CPU = types.CPUMap{"1": 10, "2": 2} - n.InitCPU = types.CPUMap{"1": 10, "2": 10} - setOpts.DeltaCPU = types.CPUMap{"1": -10, "2": -1, "3": 10} - n, err = c.SetNode(ctx, setOpts) - assert.NoError(t, err) - _, ok := n.CPU["1"] - assert.False(t, ok) - assert.Equal(t, n.CPU["2"], int64(1)) - assert.Equal(t, n.InitCPU["2"], int64(9)) - assert.Equal(t, n.CPU["3"], int64(10)) - assert.Equal(t, n.InitCPU["3"], int64(10)) - assert.Equal(t, len(n.CPU), 2) - assert.Equal(t, len(n.InitCPU), 2) - // succ set volume - n.Volume = types.VolumeMap{"/sda1": 10, "/sda2": 20} - setOpts.DeltaCPU = nil - setOpts.DeltaVolume = types.VolumeMap{"/sda0": 5, "/sda1": 0, "/sda2": -1} - n, err = c.SetNode(ctx, setOpts) - assert.NoError(t, err) - _, ok = n.Volume["/sda1"] - assert.False(t, ok) - assert.Equal(t, n.Volume["/sda0"], int64(5)) - assert.Equal(t, n.Volume["/sda2"], int64(19)) } func TestFilterNodes(t *testing.T) { assert := assert.New(t) c := NewTestCluster() + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{ + Capacity: types.NodeResourceArgs{}, + Usage: types.NodeResourceArgs{}, + }, + }, nil) store := c.store.(*storemocks.Store) nodes := []*types.Node{ { diff --git a/cluster/calcium/pod_test.go b/cluster/calcium/pod_test.go index 751410eaf..0ae2e0d4b 100644 --- a/cluster/calcium/pod_test.go +++ b/cluster/calcium/pod_test.go @@ -5,6 +5,8 @@ import ( "testing" lockmocks "github.com/projecteru2/core/lock/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" @@ -36,6 +38,11 @@ func TestAddPod(t *testing.T) { func TestRemovePod(t *testing.T) { c := NewTestCluster() ctx := context.Background() + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) assert.Error(t, c.RemovePod(ctx, "")) diff --git a/cluster/calcium/realloc.go b/cluster/calcium/realloc.go index 963ef5b3b..bb316afeb 100644 --- a/cluster/calcium/realloc.go +++ b/cluster/calcium/realloc.go @@ -3,11 +3,10 @@ package calcium import ( "context" - "github.com/projecteru2/core/engine" + "github.com/sanity-io/litter" + enginetypes "github.com/projecteru2/core/engine/types" "github.com/projecteru2/core/log" - "github.com/projecteru2/core/resources" - resourcetypes "github.com/projecteru2/core/resources/types" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" @@ -17,157 +16,62 @@ import ( // ReallocResource updates workload resource dynamically func (c *Calcium) ReallocResource(ctx context.Context, opts *types.ReallocOptions) (err error) { logger := log.WithField("Calcium", "ReallocResource").WithField("opts", opts) + log.Infof(ctx, "[ReallocResource] realloc workload %v with options %v", opts.ID, opts.ResourceOpts) workload, err := c.GetWorkload(ctx, opts.ID) if err != nil { return } + // copy origin workload + originWorkload := *workload return c.withNodePodLocked(ctx, workload.Nodename, func(ctx context.Context, node *types.Node) error { - return c.withWorkloadLocked(ctx, opts.ID, func(ctx context.Context, workload *types.Workload) error { - rrs, err := resources.MakeRequests( - types.ResourceOptions{ - CPUQuotaRequest: utils.Round(workload.CPUQuotaRequest + opts.ResourceOpts.CPUQuotaRequest), - CPUQuotaLimit: utils.Round(workload.CPUQuotaLimit + opts.ResourceOpts.CPUQuotaLimit), - CPUBind: types.ParseTriOption(opts.CPUBindOpts, len(workload.CPU) > 0), - CPU: workload.CPU, - MemoryRequest: workload.MemoryRequest + opts.ResourceOpts.MemoryRequest, - MemoryLimit: workload.MemoryLimit + opts.ResourceOpts.MemoryLimit, - StorageRequest: workload.StorageRequest + opts.ResourceOpts.StorageRequest, - StorageLimit: workload.StorageLimit + opts.ResourceOpts.StorageLimit, - VolumeRequest: types.MergeVolumeBindings(workload.VolumeRequest, opts.ResourceOpts.VolumeRequest), - VolumeLimit: types.MergeVolumeBindings(workload.VolumeLimit, opts.ResourceOpts.VolumeLimit), - VolumeExist: workload.VolumePlanRequest, - }, - ) - if err != nil { - return logger.Err(ctx, err) - } - return logger.Err(ctx, c.doReallocOnNode(ctx, node, workload, rrs)) + return logger.Err(ctx, c.doReallocOnNode(ctx, node, workload, originWorkload, opts)) }) }) } -// transaction: node resource -func (c *Calcium) doReallocOnNode(ctx context.Context, node *types.Node, workload *types.Workload, rrs resourcetypes.ResourceRequests) (err error) { - node.RecycleResources(&workload.ResourceMeta) - plans, err := resources.SelectNodesByResourceRequests(ctx, rrs, map[string]*types.Node{node.Name: node}) - if err != nil { - return err - } +func (c *Calcium) doReallocOnNode(ctx context.Context, node *types.Node, workload *types.Workload, originWorkload types.Workload, opts *types.ReallocOptions) error { + var resourceArgs map[string]types.WorkloadResourceArgs + var deltaResourceArgs map[string]types.WorkloadResourceArgs + var engineArgs types.EngineArgs + var err error - originalWorkload := *workload - resourceMeta := &types.ResourceMeta{} - if err = utils.Txn( + err = utils.Txn( ctx, - - // if update workload resources - func(ctx context.Context) (err error) { - resourceMeta := &types.ResourceMeta{} - for _, plan := range plans { - if resourceMeta, err = plan.Dispense(resourcetypes.DispenseOptions{ - Node: node, - }, resourceMeta); err != nil { - return err - } - } - - return c.doReallocWorkloadsOnInstance(ctx, node.Engine, resourceMeta, workload) - }, - // then commit changes + // if: update workload resource func(ctx context.Context) error { - for _, plan := range plans { - plan.ApplyChangesOnNode(node, 0) - } - return errors.WithStack(c.store.UpdateNodes(ctx, node)) - }, - // no need rollback - func(ctx context.Context, failureByCond bool) (err error) { - if failureByCond { - return - } - r := &types.ResourceMeta{ - CPUQuotaRequest: originalWorkload.CPUQuotaRequest, - CPUQuotaLimit: originalWorkload.CPUQuotaLimit, - CPU: originalWorkload.CPU, - NUMANode: originalWorkload.NUMANode, - MemoryRequest: originalWorkload.MemoryRequest, - MemoryLimit: originalWorkload.MemoryLimit, - VolumeRequest: originalWorkload.VolumeRequest, - VolumeLimit: originalWorkload.VolumeLimit, - VolumePlanRequest: originalWorkload.VolumePlanRequest, - VolumePlanLimit: originalWorkload.VolumePlanLimit, - VolumeChanged: resourceMeta.VolumeChanged, - StorageRequest: originalWorkload.StorageRequest, - StorageLimit: originalWorkload.StorageLimit, - } - return c.doReallocWorkloadsOnInstance(ctx, node.Engine, r, workload) - }, - - c.config.GlobalTimeout, - ); err != nil { - return - } - - go c.doRemapResourceAndLog(ctx, log.WithField("Calcium", "doReallocOnNode"), node) - return nil -} - -func (c *Calcium) doReallocWorkloadsOnInstance(ctx context.Context, engine engine.API, resourceMeta *types.ResourceMeta, workload *types.Workload) (err error) { - - originalWorkload := *workload - return utils.Txn( - ctx, - - // if: update workload resources - func(ctx context.Context) error { - r := &enginetypes.VirtualizationResource{ - CPU: resourceMeta.CPU, - Quota: resourceMeta.CPUQuotaLimit, - NUMANode: resourceMeta.NUMANode, - Memory: resourceMeta.MemoryLimit, - Volumes: resourceMeta.VolumeLimit.ToStringSlice(false, false), - VolumePlan: resourceMeta.VolumePlanLimit.ToLiteral(), - VolumeChanged: resourceMeta.VolumeChanged, - Storage: resourceMeta.StorageLimit, + // note here will change the node resource meta (stored in resource plugin) + // todo: add wal here + engineArgs, deltaResourceArgs, resourceArgs, err = c.resource.Realloc(ctx, workload.Nodename, workload.ResourceArgs, opts.ResourceOpts) + if err != nil { + return err } - return errors.WithStack(engine.VirtualizationUpdateResource(ctx, workload.ID, r)) + log.Debugf(ctx, "[doReallocOnNode] realloc workload %v, resource args %v, engine args %v", workload.ID, litter.Sdump(resourceArgs), litter.Sdump(engineArgs)) + workload.EngineArgs = engineArgs + workload.ResourceArgs = resourceArgs + return c.store.UpdateWorkload(ctx, workload) }, - - // then: update workload meta + // then: update virtualization func(ctx context.Context) error { - workload.CPUQuotaRequest = resourceMeta.CPUQuotaRequest - workload.CPUQuotaLimit = resourceMeta.CPUQuotaLimit - workload.CPU = resourceMeta.CPU - workload.NUMANode = resourceMeta.NUMANode - workload.MemoryRequest = resourceMeta.MemoryRequest - workload.MemoryLimit = resourceMeta.MemoryLimit - workload.VolumeRequest = resourceMeta.VolumeRequest - workload.VolumePlanRequest = resourceMeta.VolumePlanRequest - workload.VolumeLimit = resourceMeta.VolumeLimit - workload.VolumePlanLimit = resourceMeta.VolumePlanLimit - workload.StorageRequest = resourceMeta.StorageRequest - workload.StorageLimit = resourceMeta.StorageLimit - return errors.WithStack(c.store.UpdateWorkload(ctx, workload)) + return node.Engine.VirtualizationUpdateResource(ctx, opts.ID, &enginetypes.VirtualizationResource{EngineArgs: engineArgs}) }, - - // rollback: workload meta + // rollback: revert the resource changes and rollback workload meta func(ctx context.Context, failureByCond bool) error { if failureByCond { return nil } - r := &enginetypes.VirtualizationResource{ - CPU: originalWorkload.CPU, - Quota: originalWorkload.CPUQuotaLimit, - NUMANode: originalWorkload.NUMANode, - Memory: originalWorkload.MemoryLimit, - Volumes: originalWorkload.VolumeLimit.ToStringSlice(false, false), - VolumePlan: originalWorkload.VolumePlanLimit.ToLiteral(), - VolumeChanged: resourceMeta.VolumeChanged, - Storage: originalWorkload.StorageLimit, + if err := c.resource.RollbackRealloc(ctx, workload.Nodename, deltaResourceArgs); err != nil { + log.Errorf(ctx, "[doReallocOnNode] failed to rollback workload %v, resource args %v, engine args %v, err %v", workload.ID, litter.Sdump(resourceArgs), litter.Sdump(engineArgs), err) + // don't return here, so the node resource can still be fixed } - return errors.WithStack(engine.VirtualizationUpdateResource(ctx, workload.ID, r)) + return errors.WithStack(c.store.UpdateWorkload(ctx, &originWorkload)) }, - c.config.GlobalTimeout, ) + if err != nil { + return errors.WithStack(err) + } + go c.SendNodeMetrics(ctx, node.Name) + go c.doRemapResourceAndLog(ctx, log.WithField("Calcium", "doReallocOnNode"), node) + return nil } diff --git a/cluster/calcium/realloc_test.go b/cluster/calcium/realloc_test.go index 278f26bc0..a8faada82 100644 --- a/cluster/calcium/realloc_test.go +++ b/cluster/calcium/realloc_test.go @@ -3,41 +3,26 @@ package calcium import ( "context" "testing" - "time" enginemocks "github.com/projecteru2/core/engine/mocks" enginetypes "github.com/projecteru2/core/engine/types" lockmocks "github.com/projecteru2/core/lock/mocks" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - complexscheduler "github.com/projecteru2/core/scheduler/complex" - schedulermocks "github.com/projecteru2/core/scheduler/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" - "github.com/docker/go-units" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" ) -func newReallocOptions(id string, cpu float64, memory int64, vbs types.VolumeBindings, bindCPUOpt, memoryLimitOpt types.TriOptions) *types.ReallocOptions { - return &types.ReallocOptions{ - ID: id, - CPUBindOpts: bindCPUOpt, - ResourceOpts: types.ResourceOptions{ - CPUQuotaLimit: cpu, - MemoryLimit: memory, - VolumeLimit: vbs, - }, - } -} - func TestRealloc(t *testing.T) { c := NewTestCluster() ctx := context.Background() store := &storemocks.Store{} c.store = store c.config.Scheduler.ShareBase = 100 + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) lock := &lockmocks.DistributedLock{} lock.On("Lock", mock.Anything).Return(context.TODO(), nil) @@ -48,14 +33,8 @@ func TestRealloc(t *testing.T) { node1 := &types.Node{ NodeMeta: types.NodeMeta{ - Name: "node1", - MemCap: int64(units.GiB), - CPU: types.CPUMap{"0": 10, "1": 70, "2": 10, "3": 100}, - InitCPU: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100}, - Endpoint: "http://1.1.1.1:1", - NUMA: types.NUMA{"2": "0"}, - NUMAMemory: types.NUMAMemory{"0": 100000}, - Volume: types.VolumeMap{"/dir0": 100}, + Name: "node1", + Endpoint: "http://1.1.1.1:1", }, Engine: engine, } @@ -63,322 +42,67 @@ func TestRealloc(t *testing.T) { newC1 := func(context.Context, []string) []*types.Workload { return []*types.Workload{ { - 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", - }, - } - } - - newC2 := func(context.Context, []string) []*types.Workload { - return []*types.Workload{ - { - 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", + ID: "c1", + Podname: "p1", + Engine: engine, + ResourceArgs: types.ResourceMeta{}, + Nodename: "node1", }, } } store.On("GetWorkload", mock.Anything, "c1").Return(newC1(context.TODO(), nil)[0], nil) + opts := &types.ReallocOptions{ + ID: "c1", + ResourceOpts: types.WorkloadResourceOpts{}, + } // failed by GetNode store.On("GetNode", mock.Anything, "node1").Return(nil, types.ErrNoETCD).Once() - err := c.ReallocResource(ctx, newReallocOptions("c1", 0.1, 2*int64(units.GiB), nil, types.TriKeep, types.TriKeep)) + err := c.ReallocResource(ctx, opts) assert.EqualError(t, err, "ETCD must be set") store.AssertExpectations(t) store.On("GetNode", mock.Anything, "node1").Return(node1, nil) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{ + Capacity: types.NodeResourceArgs{}, + Usage: types.NodeResourceArgs{}, + }, + }, nil) // failed by lock store.On("CreateLock", mock.Anything, mock.Anything).Return(nil, types.ErrNoETCD).Once() - err = c.ReallocResource(ctx, newReallocOptions("c1", -1, 2*int64(units.GiB), nil, types.TriKeep, types.TriKeep)) + err = c.ReallocResource(ctx, opts) assert.EqualError(t, err, "ETCD must be set") store.AssertExpectations(t) store.On("CreateLock", mock.Anything, mock.Anything).Return(lock, nil) - - // failed by newCPU < 0 store.On("GetWorkloads", mock.Anything, []string{"c1"}).Return(newC1, nil) - 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 no enough mem - simpleMockScheduler := &schedulermocks.Scheduler{} - scheduler.InitSchedulerV1(simpleMockScheduler) - c.scheduler = simpleMockScheduler - simpleMockScheduler.On("ReselectCPUNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(resourcetypes.ScheduleInfo{}, nil, 0, types.ErrInsufficientMEM).Once() - 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) - - // failed by wrong total - simpleMockScheduler.On("ReselectCPUNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(resourcetypes.ScheduleInfo{}, nil, 0, nil).Once() - scheduleInfos := []resourcetypes.ScheduleInfo{{ - NodeMeta: types.NodeMeta{Name: "node1"}, - Capacity: 2, - }} - simpleMockScheduler.On("SelectStorageNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos, 100, nil) - nodeVolumePlans := map[string][]types.VolumePlan{ - "node1": {{types.MustToVolumeBinding("AUTO:/data:rw:50"): types.VolumeMap{"/dir0": 50}}}, - } - simpleMockScheduler.On("ReselectVolumeNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(resourcetypes.ScheduleInfo{}, nil, 1, nil).Once() - err = c.ReallocResource(ctx, newReallocOptions("c1", 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) - assert.EqualError(t, err, "cannot alloc a each node plan, not enough capacity") - simpleMockScheduler.AssertExpectations(t) - store.AssertExpectations(t) - - // vaild cpu plans - nodeCPUPlans := map[string][]types.CPUMap{ - node1.Name: { - {"3": 100}, - {"2": 100}, - {"node1": 100}, - }, - } - simpleMockScheduler.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos, 2, nil).Once() - simpleMockScheduler.On("SelectVolumeNodes", mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos, nil, 100, nil).Once() - // failed by apply resource - engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(types.ErrBadWorkloadID).Once() - // reset node - node1 = &types.Node{ - NodeMeta: types.NodeMeta{ - Name: "node1", - MemCap: int64(units.GiB), - CPU: types.CPUMap{"0": 10, "1": 70, "2": 10, "3": 100}, - Endpoint: "http://1.1.1.1:1", - }, - Engine: engine, - } - store.On("GetWorkload", mock.Anything, "c2").Return(newC2(nil, nil)[0], nil) - store.On("GetWorkloads", 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, "workload 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 workload - simpleMockScheduler.On("ReselectCPUNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos[0], nodeCPUPlans, 2, nil).Once() - engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(nil) - store.On("UpdateWorkload", mock.Anything, mock.Anything).Return(types.ErrBadWorkloadID).Times(1) - simpleMockScheduler.On("ReselectVolumeNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos[0], nil, 100, nil).Once() - err = c.ReallocResource(ctx, newReallocOptions("c1", 0.1, 2*int64(units.MiB), nil, types.TriKeep, types.TriKeep)) - assert.EqualError(t, err, "workload ID must be length of 64") - simpleMockScheduler.AssertExpectations(t) - store.AssertExpectations(t) - - // failed by volume schedule error - simpleMockScheduler.On("ReselectCPUNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(resourcetypes.ScheduleInfo{}, nodeCPUPlans, 2, nil).Once() - simpleMockScheduler.On("ReselectVolumeNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos[0], nil, 2, nil).Once() - 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 each node plan, not enough capacity") - simpleMockScheduler.AssertExpectations(t) - store.AssertExpectations(t) - - // good to go - // rest everything - scheduleInfos[0].NodeMeta.Name = "node2" - node2 := &types.Node{ - NodeMeta: types.NodeMeta{ - Name: "node2", - MemCap: int64(units.GiB), - CPU: types.CPUMap{"0": 10, "1": 70, "2": 10, "3": 100}, - InitCPU: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100}, - Endpoint: "http://1.1.1.1:1", - NUMA: types.NUMA{"2": "0"}, - NUMAMemory: types.NUMAMemory{"0": 100000}, - Volume: types.VolumeMap{"/dir0": 200, "/dir1": 200, "/dir2": 200}, - }, - VolumeUsed: int64(300), - Engine: engine, - } - c3 := &types.Workload{ - 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", - } - nodeCPUPlans = map[string][]types.CPUMap{ - node2.Name: { - {"3": 100}, - }, - } - nodeVolumePlans = map[string][]types.VolumePlan{ - node2.Name: { - { - types.MustToVolumeBinding("AUTO:/data0:rw:50"): types.VolumeMap{"/dir1": 50}, - types.MustToVolumeBinding("AUTO:/data1:rw:200"): types.VolumeMap{"/dir2": 200}, - }, - { - types.MustToVolumeBinding("AUTO:/data0:rw:50"): types.VolumeMap{"/dir0": 50}, - types.MustToVolumeBinding("AUTO:/data1:rw:200"): types.VolumeMap{"/dir1": 200}, - }, - }, - } - simpleMockScheduler.On("ReselectCPUNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos[0], nodeCPUPlans, 2, nil) - simpleMockScheduler.On("ReselectVolumeNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos[0], nodeVolumePlans, 2, nil).Once() - store.On("GetNode", mock.Anything, "node2").Return(node2, nil) - store.On("GetWorkload", mock.Anything, "c3").Return(c3, nil) - store.On("GetWorkloads", mock.Anything, []string{"c3"}).Return([]*types.Workload{c3}, nil) - store.On("UpdateWorkload", mock.Anything, mock.Anything).Return(types.ErrBadWorkloadID).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, "workload ID must be length of 64") - assert.Equal(t, node2.CPU["3"], int64(100)) - 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)) + // failed by plugin + plugin.On("GetReallocArgs", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, context.DeadlineExceeded).Once() + err = c.ReallocResource(ctx, opts) + assert.Error(t, err) + plugin.On("GetReallocArgs", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetReallocArgsResponse{}, nil) + plugin.On("SetNodeResourceUsage", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.SetNodeResourceUsageResponse{ + Before: types.NodeResourceArgs{}, + After: types.NodeResourceArgs{}, + }, nil) + + // failed by UpdateWorkload + store.On("UpdateWorkload", mock.Anything, mock.Anything).Return(types.ErrNoETCD).Once() + err = c.ReallocResource(ctx, opts) + assert.EqualError(t, err, "ETCD must be set") store.AssertExpectations(t) - simpleMockScheduler.AssertExpectations(t) -} - -func TestReallocBindCpu(t *testing.T) { - c := NewTestCluster() - ctx := context.Background() - store := &storemocks.Store{} - c.store = store - - lock := &lockmocks.DistributedLock{} - lock.On("Lock", mock.Anything).Return(context.TODO(), nil) - lock.On("Unlock", mock.Anything).Return(nil) - store.On("CreateLock", mock.Anything, mock.Anything).Return(lock, nil) - engine := &enginemocks.API{} - engine.On("VirtualizationInspect", mock.Anything, mock.Anything).Return(&enginetypes.VirtualizationInfo{}, nil) - - config := types.Config{ - LogLevel: "", - Bind: "", - LockTimeout: 0, - GlobalTimeout: 0, - Statsd: "", - Profile: "", - CertPath: "", - Auth: types.AuthConfig{}, - GRPCConfig: types.GRPCConfig{}, - Git: types.GitConfig{}, - Etcd: types.EtcdConfig{}, - Docker: types.DockerConfig{}, - Scheduler: types.SchedConfig{MaxShare: -1, ShareBase: 100}, - Virt: types.VirtConfig{}, - Systemd: types.SystemdConfig{}, - } - simpleMockScheduler, _ := complexscheduler.New(config) - c.scheduler = simpleMockScheduler - scheduler.InitSchedulerV1(simpleMockScheduler) - - //test bindCpu - node3 := &types.Node{ - NodeMeta: types.NodeMeta{ - Name: "node3", - MemCap: int64(units.GiB), - CPU: types.CPUMap{"0": 10, "1": 70, "2": 10, "3": 100}, - InitCPU: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100}, - Endpoint: "http://1.1.1.1:1", - NUMA: types.NUMA{"2": "0"}, - NUMAMemory: types.NUMAMemory{"0": 100000}, - Volume: types.VolumeMap{"/dir0": 200, "/dir1": 200, "/dir2": 200}, - }, - VolumeUsed: int64(300), - CPUUsed: 2.1, - Engine: engine, - } - c5 := &types.Workload{ - 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.Workload{ - 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) - store.On("GetWorkload", mock.Anything, "c5").Return(c5, nil) - store.On("GetWorkloads", mock.Anything, []string{"c5"}).Return([]*types.Workload{c5}, nil) - engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(nil) store.On("UpdateWorkload", mock.Anything, mock.Anything).Return(nil) - // failed by UpdateNodes - store.On("UpdateNodes", mock.Anything, mock.Anything).Return(types.ErrBadWorkloadID).Once() - err := c.ReallocResource(ctx, newReallocOptions("c5", 0.1, 2*int64(units.MiB), nil, types.TriFalse, types.TriKeep)) - assert.Error(t, err) - store.AssertExpectations(t) + // failed by virtualization update resource + engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(types.ErrNilEngine).Once() + err = c.ReallocResource(ctx, opts) + assert.ErrorIs(t, err, types.ErrNilEngine) + engine.On("VirtualizationUpdateResource", mock.Anything, mock.Anything, mock.Anything).Return(nil) + // success store.On("ListNodeWorkloads", mock.Anything, mock.Anything, mock.Anything).Return(nil, types.ErrNoETCD) - store.On("UpdateNodes", mock.Anything, mock.Anything).Return(nil) - 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.ResourceMeta.CPU)) - time.Sleep(time.Second) - store.AssertExpectations(t) - - store.On("GetWorkload", mock.Anything, "c6").Return(c6, nil) - store.On("GetWorkloads", mock.Anything, []string{"c6"}).Return([]*types.Workload{c6}, nil) - err = c.ReallocResource(ctx, newReallocOptions("c6", 0.1, 2*int64(units.MiB), nil, types.TriTrue, types.TriKeep)) - assert.NoError(t, err) - assert.NotEmpty(t, c6.ResourceMeta.CPU) - store.AssertExpectations(t) - - node3.CPU = types.CPUMap{"0": 10, "1": 70, "2": 100, "3": 100} - err = c.ReallocResource(ctx, newReallocOptions("c5", -0.1, 2*int64(units.MiB), nil, types.TriTrue, types.TriKeep)) - assert.NoError(t, err) - 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(c6.ResourceMeta.CPU)) - store.AssertExpectations(t) + err = c.ReallocResource(ctx, opts) + assert.Nil(t, err) } diff --git a/cluster/calcium/remove.go b/cluster/calcium/remove.go index 2349c8817..51c1a8209 100644 --- a/cluster/calcium/remove.go +++ b/cluster/calcium/remove.go @@ -6,6 +6,7 @@ import ( "sync" "github.com/projecteru2/core/log" + "github.com/projecteru2/core/resources" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" @@ -41,7 +42,12 @@ func (c *Calcium) RemoveWorkload(ctx context.Context, ids []string, force bool) ctx, // if func(ctx context.Context) error { - return errors.WithStack(c.store.UpdateNodeResource(ctx, node, &workload.ResourceMeta, types.ActionIncr)) + resourceArgs := map[string]types.WorkloadResourceArgs{} + for plugin, args := range workload.ResourceArgs { + resourceArgs[plugin] = args + } + _, _, err = c.resource.SetNodeResourceUsage(ctx, node.Name, nil, nil, []map[string]types.WorkloadResourceArgs{resourceArgs}, true, resources.Decr) + return errors.WithStack(err) }, // then func(ctx context.Context) (err error) { @@ -55,7 +61,12 @@ func (c *Calcium) RemoveWorkload(ctx context.Context, ids []string, force bool) if failedByCond { return nil } - return errors.WithStack(c.store.UpdateNodeResource(ctx, node, &workload.ResourceMeta, types.ActionDecr)) + resourceArgs := map[string]types.WorkloadResourceArgs{} + for plugin, args := range workload.ResourceArgs { + resourceArgs[plugin] = args + } + _, _, err = c.resource.SetNodeResourceUsage(ctx, node.Name, nil, nil, []map[string]types.WorkloadResourceArgs{resourceArgs}, true, resources.Incr) + return errors.WithStack(err) }, c.config.GlobalTimeout, ) @@ -66,6 +77,7 @@ func (c *Calcium) RemoveWorkload(ctx context.Context, ids []string, force bool) } ch <- ret } + go c.SendNodeMetrics(ctx, node.Name) go c.doRemapResourceAndLog(ctx, logger, node) return nil }); err != nil { diff --git a/cluster/calcium/remove_test.go b/cluster/calcium/remove_test.go index ceece2358..4b5b7f504 100644 --- a/cluster/calcium/remove_test.go +++ b/cluster/calcium/remove_test.go @@ -7,6 +7,8 @@ import ( enginemocks "github.com/projecteru2/core/engine/mocks" lockmocks "github.com/projecteru2/core/lock/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" @@ -22,6 +24,14 @@ func TestRemoveWorkload(t *testing.T) { lock.On("Lock", mock.Anything).Return(context.TODO(), nil) lock.On("Unlock", mock.Anything).Return(nil) store := c.store.(*storemocks.Store) + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + }, nil) + plugin.On("SetNodeResourceUsage", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.SetNodeResourceUsageResponse{ + Before: types.NodeResourceArgs{}, + After: types.NodeResourceArgs{}, + }, nil) // failed by GetWorkload store.On("GetWorkloads", mock.Anything, mock.Anything).Return(nil, types.ErrNoETCD).Once() @@ -42,6 +52,7 @@ func TestRemoveWorkload(t *testing.T) { for r := range ch { assert.False(t, r.Success) } + time.Sleep(time.Second) store.AssertExpectations(t) // failed by Remove @@ -51,7 +62,7 @@ func TestRemoveWorkload(t *testing.T) { Name: "test", }, } - store.On("UpdateNodeResource", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) + plugin.On("SetNodeResourceUsage", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, nil).Twice() store.On("GetNode", mock.Anything, mock.Anything).Return(node, nil) store.On("RemoveWorkload", mock.Anything, mock.Anything).Return(types.ErrNoETCD).Twice() store.On("ListNodeWorkloads", mock.Anything, mock.Anything, mock.Anything).Return(nil, types.ErrNoETCD) @@ -70,7 +81,6 @@ func TestRemoveWorkload(t *testing.T) { engine.On("VirtualizationRemove", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) store.On("GetWorkloads", mock.Anything, mock.Anything).Return([]*types.Workload{workload}, nil) store.On("RemoveWorkload", mock.Anything, mock.Anything).Return(nil) - store.On("UpdateNodeResource", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) ch, err = c.RemoveWorkload(ctx, []string{"xx"}, false) assert.NoError(t, err) for r := range ch { diff --git a/cluster/calcium/replace.go b/cluster/calcium/replace.go index a2700bc3f..60edba185 100644 --- a/cluster/calcium/replace.go +++ b/cluster/calcium/replace.go @@ -61,17 +61,7 @@ func (c *Calcium) ReplaceWorkload(ctx context.Context, opts *types.ReplaceOption } // 使用复制之后的配置 // 停老的,起新的 - replaceOpts.ResourceOpts = types.ResourceOptions{ - CPUQuotaRequest: workload.CPUQuotaRequest, - CPUQuotaLimit: workload.CPUQuotaLimit, - CPUBind: len(workload.CPU) > 0, - MemoryRequest: workload.MemoryRequest, - MemoryLimit: workload.MemoryLimit, - StorageRequest: workload.StorageRequest, - StorageLimit: workload.StorageLimit, - VolumeRequest: workload.VolumeRequest, - VolumeLimit: workload.VolumeLimit, - } + // replaceOpts.ResourceOpts = workload.ResourceUsage // 覆盖 podname 如果做全量更新的话 replaceOpts.Podname = workload.Podname // 覆盖 Volumes @@ -146,21 +136,15 @@ func (c *Calcium) doReplaceWorkload( }) } + // copy resource args createMessage := &types.CreateWorkloadMessage{ - ResourceMeta: types.ResourceMeta{ - MemoryRequest: workload.MemoryRequest, - MemoryLimit: workload.MemoryLimit, - StorageRequest: workload.StorageRequest, - StorageLimit: workload.StorageLimit, - CPUQuotaRequest: workload.CPUQuotaRequest, - CPUQuotaLimit: workload.CPUQuotaLimit, - CPU: workload.CPU, - VolumeRequest: workload.VolumeRequest, - VolumePlanRequest: workload.VolumePlanRequest, - VolumeLimit: workload.VolumeLimit, - VolumePlanLimit: workload.VolumePlanLimit, - }, + ResourceArgs: map[string]types.WorkloadResourceArgs{}, + EngineArgs: workload.EngineArgs, } + for plugin, rawParams := range workload.ResourceArgs { + createMessage.ResourceArgs[plugin] = rawParams + } + if err = utils.Txn( ctx, // if diff --git a/cluster/calcium/replace_test.go b/cluster/calcium/replace_test.go index 87ba2fa27..c19766086 100644 --- a/cluster/calcium/replace_test.go +++ b/cluster/calcium/replace_test.go @@ -7,6 +7,8 @@ import ( enginemocks "github.com/projecteru2/core/engine/mocks" enginetypes "github.com/projecteru2/core/engine/types" lockmocks "github.com/projecteru2/core/lock/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" @@ -21,6 +23,10 @@ func TestReplaceWorkload(t *testing.T) { lock.On("Lock", mock.Anything).Return(context.TODO(), nil) lock.On("Unlock", mock.Anything).Return(nil) store := c.store.(*storemocks.Store) + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + }, nil) _, err := c.ReplaceWorkload(ctx, &types.ReplaceOptions{ DeployOptions: types.DeployOptions{ diff --git a/cluster/calcium/resource.go b/cluster/calcium/resource.go index 3bb0f055e..436bd92c5 100644 --- a/cluster/calcium/resource.go +++ b/cluster/calcium/resource.go @@ -6,7 +6,6 @@ import ( enginetypes "github.com/projecteru2/core/engine/types" "github.com/projecteru2/core/log" - resourcetypes "github.com/projecteru2/core/resources/types" "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" @@ -65,172 +64,105 @@ func (c *Calcium) NodeResource(ctx context.Context, nodename string, fix bool) ( func (c *Calcium) doGetNodeResource(ctx context.Context, nodename string, fix bool) (*types.NodeResource, error) { var nr *types.NodeResource return nr, c.withNodePodLocked(ctx, nodename, func(ctx context.Context, node *types.Node) error { - workloads, err := c.ListNodeWorkloads(ctx, node.Name, nil) + workloads, err := c.ListNodeWorkloads(ctx, nodename, nil) if err != nil { + log.Errorf(ctx, "[doGetNodeResource] failed to list node workloads, node %v, err: %v", nodename, err) return err } - nr = &types.NodeResource{ - Name: node.Name, CPU: node.CPU, MemCap: node.MemCap, StorageCap: node.StorageCap, - Workloads: workloads, Diffs: []string{}, - } - - cpuByWorkloads := 0.0 - memoryByWorkloads := int64(0) - storageByWorkloads := int64(0) // volume inclusive - cpumapByWorkloads := types.CPUMap{} - volumeByWorkloads := int64(0) - volumeMapByWorkloads := types.VolumeMap{} - monopolyVolumeByWorkloads := map[string][]string{} // volume -> array of workload id - for _, workload := range workloads { - cpuByWorkloads = utils.Round(cpuByWorkloads + workload.CPUQuotaRequest) - memoryByWorkloads += workload.MemoryRequest - storageByWorkloads += workload.StorageRequest - cpumapByWorkloads.Add(workload.CPU) - for vb, vmap := range workload.VolumePlanRequest { - volumeByWorkloads += vmap.Total() - volumeMapByWorkloads.Add(vmap) - if vb.RequireScheduleMonopoly() { - monopolyVolumeByWorkloads[vmap.GetResourceID()] = append(monopolyVolumeByWorkloads[vmap.GetResourceID()], workload.ID) - } - } - } - nr.CPUPercent = cpuByWorkloads / float64(len(node.InitCPU)) - nr.MemoryPercent = float64(memoryByWorkloads) / float64(node.InitMemCap) - nr.NUMAMemoryPercent = map[string]float64{} - nr.VolumePercent = float64(node.VolumeUsed) / float64(node.InitVolume.Total()) - for nodeID, nmemory := range node.NUMAMemory { - if initMemory, ok := node.InitNUMAMemory[nodeID]; ok { - nr.NUMAMemoryPercent[nodeID] = float64(nmemory) / float64(initMemory) - } - } - - // cpu - if cpuByWorkloads != node.CPUUsed { - nr.Diffs = append(nr.Diffs, - fmt.Sprintf("node.CPUUsed != sum(workload.CPURequest): %.2f != %.2f", node.CPUUsed, cpuByWorkloads)) - } - node.CPU.Add(cpumapByWorkloads) - for i, v := range node.CPU { - if node.InitCPU[i] != v { - nr.Diffs = append(nr.Diffs, - fmt.Sprintf("\tsum(workload.CPU[%s]) + node.CPU[%s] != node.InitCPU[%s]: %d + %d != %d", i, i, i, - cpumapByWorkloads[i], node.CPU[i]-cpumapByWorkloads[i], node.InitCPU[i])) - } - } - - // memory - if memoryByWorkloads+node.MemCap != node.InitMemCap { - nr.Diffs = append(nr.Diffs, - fmt.Sprintf("node.MemCap + sum(workload.memoryRequest) != node.InitMemCap: %d + %d != %d", - node.MemCap, memoryByWorkloads, node.InitMemCap)) - } - // storage - nr.StoragePercent = 0 - if node.InitStorageCap != 0 { - nr.StoragePercent = float64(storageByWorkloads) / float64(node.InitStorageCap) - } - if storageByWorkloads+node.StorageCap != node.InitStorageCap { - nr.Diffs = append(nr.Diffs, fmt.Sprintf("sum(workload.storageRequest) + node.StorageCap != node.InitStorageCap: %d + %d != %d", storageByWorkloads, node.StorageCap, node.InitStorageCap)) - } - - // volume - if node.VolumeUsed != volumeByWorkloads { - nr.Diffs = append(nr.Diffs, fmt.Sprintf("node.VolumeUsed != sum(workload.VolumeRequest): %d != %d", node.VolumeUsed, volumeByWorkloads)) - } - node.Volume.Add(volumeMapByWorkloads) - for i, v := range node.Volume { - if node.InitVolume[i] != v { - nr.Diffs = append(nr.Diffs, fmt.Sprintf("\tsum(workload.Volume[%s]) + node.Volume[%s] != node.InitVolume[%s]: %d + %d != %d", - i, i, i, - volumeMapByWorkloads[i], node.Volume[i]-volumeMapByWorkloads[i], node.InitVolume[i])) - } - } - for vol, ids := range monopolyVolumeByWorkloads { - idx := utils.Unique(ids, func(i int) string { return ids[i] }) - if len(ids[:idx]) > 1 { - nr.Diffs = append(nr.Diffs, fmt.Sprintf("\tmonopoly volume used by multiple workloads: %s, %+v", vol, ids)) - } - } - - // volume and storage - if node.InitStorageCap < node.InitVolume.Total() { - nr.Diffs = append(nr.Diffs, fmt.Sprintf("init storage < init volumes: %d < %d", node.InitStorageCap, node.InitVolume.Total())) - } - - if err := node.Engine.ResourceValidate(ctx, cpuByWorkloads, cpumapByWorkloads, memoryByWorkloads, storageByWorkloads); err != nil { - nr.Diffs = append(nr.Diffs, err.Error()) + // TODO: percentage? + resourceCapacity, resourceUsage, diffs, err := c.resource.GetNodeResourceInfo(ctx, nodename, workloads, fix) + if err != nil { + log.Errorf(ctx, "[doGetNodeResource] failed to get node resource, node %v, err: %v", nodename, err) + return err } if fix { - if err := c.doFixDiffResource(ctx, node, cpuByWorkloads, memoryByWorkloads, storageByWorkloads, volumeByWorkloads); err != nil { - log.Warnf(ctx, "[doGetNodeResource] fix node resource failed %v", err) - } + go c.SendNodeMetrics(ctx, node.Name) } + nr = &types.NodeResource{ + Name: nodename, + ResourceCapacity: resourceCapacity, + ResourceUsage: resourceUsage, + Diffs: diffs, + Workloads: workloads, + } return nil }) } -func (c *Calcium) doFixDiffResource(ctx context.Context, node *types.Node, cpuByWorkloads float64, memoryByWorkloads, storageByWorkloads, volumeByWorkloads int64) (err error) { - var n *types.Node - if n, err = c.GetNode(ctx, node.Name); err != nil { - return err - } - n.CPUUsed = cpuByWorkloads - for i, v := range node.CPU { - n.CPU[i] += node.InitCPU[i] - v - } - n.MemCap = node.InitMemCap - memoryByWorkloads - if n.InitStorageCap < n.InitVolume.Total() { - n.InitStorageCap = n.InitVolume.Total() - } - n.StorageCap = n.InitStorageCap - storageByWorkloads - n.VolumeUsed = volumeByWorkloads - for i, v := range node.Volume { - n.Volume[i] += node.InitVolume[i] - v +func (c *Calcium) doGetDeployMap(ctx context.Context, nodes []string, opts *types.DeployOptions) (map[string]int, error) { + // get nodes with capacity > 0 + nodeResourceInfoMap, total, err := c.resource.GetNodesDeployCapacity(ctx, nodes, opts.ResourceOpts) + if err != nil { + log.Errorf(ctx, "[doGetDeployMap] failed to select available nodes, nodes %v, err %v", nodes, err) + return nil, errors.WithStack(err) } - return errors.WithStack(c.store.UpdateNodes(ctx, n)) -} -func (c *Calcium) doAllocResource(ctx context.Context, nodeMap map[string]*types.Node, opts *types.DeployOptions) ([]resourcetypes.ResourcePlans, map[string]int, error) { - total, plans, strategyInfos, err := c.doCalculateCapacity(ctx, nodeMap, opts) + // get deployed & processing workload count on each node + deployStatusMap, err := c.store.GetDeployStatus(ctx, opts.Name, opts.Entrypoint.Name) if err != nil { - return nil, nil, err + log.Errorf(ctx, "failed to get deploy status for %v_%v, err %v", opts.Name, opts.Entrypoint.Name, err) + return nil, errors.WithStack(err) } - if err = c.store.MakeDeployStatus(ctx, opts.Name, opts.Entrypoint.Name, strategyInfos); err != nil { - return nil, nil, errors.WithStack(err) + + // generate strategy info + strategyInfos := []strategy.Info{} + for node, resourceInfo := range nodeResourceInfoMap { + strategyInfos = append(strategyInfos, strategy.Info{ + Nodename: node, + Usage: resourceInfo.Usage, + Rate: resourceInfo.Rate, + Capacity: resourceInfo.Capacity, + Count: deployStatusMap[node], + }) } - deployMap, err := strategy.Deploy(ctx, opts, strategyInfos, total) + + // generate deploy plan + deployMap, err := strategy.Deploy(ctx, opts.DeployStrategy, opts.Count, opts.NodesLimit, strategyInfos, total) if err != nil { - return nil, nil, err + return nil, errors.WithStack(err) } - log.Infof(ctx, "[Calium.doAllocResource] deployMap: %+v", deployMap) - return plans, deployMap, nil + + return deployMap, nil +} + +type remapMsg struct { + id string + err error } // called on changes of resource binding, such as cpu binding // as an internal api, remap doesn't lock node, the responsibility of that should be taken on by caller -func (c *Calcium) remapResource(ctx context.Context, node *types.Node) (ch <-chan enginetypes.VirtualizationRemapMessage, err error) { +func (c *Calcium) remapResource(ctx context.Context, node *types.Node) (ch chan *remapMsg, err error) { workloads, err := c.store.ListNodeWorkloads(ctx, node.Name, nil) if err != nil { return } - remapOpts := &enginetypes.VirtualizationRemapOptions{ - CPUAvailable: node.CPU, - CPUInit: node.InitCPU, - CPUShareBase: int64(c.config.Scheduler.ShareBase), - WorkloadResources: make(map[string]enginetypes.VirtualizationResource), - } + + workloadMap := map[string]*types.Workload{} for _, workload := range workloads { - remapOpts.WorkloadResources[workload.ID] = enginetypes.VirtualizationResource{ - CPU: workload.CPU, - Quota: workload.CPUQuotaLimit, - NUMANode: workload.NUMANode, - } + workloadMap[workload.ID] = workload } - ch, err = node.Engine.VirtualizationResourceRemap(ctx, remapOpts) - return ch, errors.WithStack(err) + + engineArgsMap, err := c.resource.GetRemapArgs(ctx, node.Name, workloadMap) + if err != nil { + return nil, err + } + + ch = make(chan *remapMsg, len(engineArgsMap)) + go func() { + defer close(ch) + for workloadID, engineArgs := range engineArgsMap { + ch <- &remapMsg{ + id: workloadID, + err: node.Engine.VirtualizationUpdateResource(ctx, workloadID, &enginetypes.VirtualizationResource{EngineArgs: engineArgs}), + } + } + }() + + return ch, nil } func (c *Calcium) doRemapResourceAndLog(ctx context.Context, logger log.Fields, node *types.Node) { @@ -242,7 +174,8 @@ func (c *Calcium) doRemapResourceAndLog(ctx context.Context, logger log.Fields, logger = logger.WithField("Calcium", "doRemapResourceAndLog").WithField("nodename", node.Name) if ch, err := c.remapResource(ctx, node); logger.Err(ctx, err) == nil { for msg := range ch { - logger.WithField("id", msg.ID).Err(ctx, msg.Error) // nolint:errcheck + log.Infof(ctx, "[doRemapResourceAndLog] id %v", msg.id) + logger.WithField("id", msg.id).Err(ctx, msg.err) // nolint:errcheck } } return nil diff --git a/cluster/calcium/resource_test.go b/cluster/calcium/resource_test.go index 79da20cc7..a1a915540 100644 --- a/cluster/calcium/resource_test.go +++ b/cluster/calcium/resource_test.go @@ -5,17 +5,14 @@ import ( "fmt" "strings" "testing" - "time" enginemocks "github.com/projecteru2/core/engine/mocks" enginetypes "github.com/projecteru2/core/engine/types" lockmocks "github.com/projecteru2/core/lock/mocks" "github.com/projecteru2/core/log" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - schedulermocks "github.com/projecteru2/core/scheduler/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" - "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" "github.com/stretchr/testify/assert" @@ -29,6 +26,12 @@ func TestPodResource(t *testing.T) { nodename := "testnode" store := &storemocks.Store{} c.store = store + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) + lock := &lockmocks.DistributedLock{} lock.On("Lock", mock.Anything).Return(context.TODO(), nil) lock.On("Unlock", mock.Anything).Return(nil) @@ -41,12 +44,7 @@ func TestPodResource(t *testing.T) { // failed by ListNodeWorkloads node := &types.Node{ NodeMeta: types.NodeMeta{ - Name: nodename, - CPU: types.CPUMap{"0": 0, "1": 10}, - MemCap: 2, - InitCPU: types.CPUMap{"0": 100, "1": 100}, - InitMemCap: 6, - InitStorageCap: 10, + Name: nodename, }, } store.On("GetNodesByPod", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*types.Node{node}, nil) @@ -61,24 +59,10 @@ func TestPodResource(t *testing.T) { workloads := []*types.Workload{ { - ResourceMeta: types.ResourceMeta{ - MemoryRequest: 1, - MemoryLimit: 1, - CPU: types.CPUMap{"0": 100, "1": 30}, - CPUQuotaRequest: 1.3, - CPUQuotaLimit: 1.3, - }, + ResourceArgs: map[string]types.WorkloadResourceArgs{}, }, { - ResourceMeta: types.ResourceMeta{ - MemoryLimit: 2, - MemoryRequest: 2, - CPU: types.CPUMap{"1": 50}, - CPUQuotaRequest: 0.5, - CPUQuotaLimit: 0.5, - StorageRequest: 1, - StorageLimit: 1, - }, + ResourceArgs: map[string]types.WorkloadResourceArgs{}, }, } store.On("ListNodeWorkloads", mock.Anything, mock.Anything, mock.Anything).Return(workloads, nil) @@ -91,9 +75,6 @@ func TestPodResource(t *testing.T) { r, err := c.PodResource(ctx, podname) assert.NoError(t, err) first := <-r - assert.Equal(t, first.CPUPercent, 0.9) - assert.Equal(t, first.MemoryPercent, 0.5) - assert.Equal(t, first.StoragePercent, 0.1) assert.NotEmpty(t, first.Diffs) } @@ -103,19 +84,22 @@ func TestNodeResource(t *testing.T) { nodename := "testnode" store := &storemocks.Store{} c.store = store + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) + plugin.On("FixNodeResource", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) lock := &lockmocks.DistributedLock{} store.On("CreateLock", mock.Anything, mock.Anything).Return(lock, nil) lock.On("Lock", mock.Anything).Return(context.TODO(), nil) lock.On("Unlock", mock.Anything).Return(nil) node := &types.Node{ NodeMeta: types.NodeMeta{ - Name: nodename, - CPU: types.CPUMap{"0": 0, "1": 10}, - MemCap: 2, - InitCPU: types.CPUMap{"0": 100, "1": 100}, - InitMemCap: 6, - NUMAMemory: types.NUMAMemory{"0": 1, "1": 1}, - InitNUMAMemory: types.NUMAMemory{"0": 3, "1": 3}, + Name: nodename, }, } engine := &enginemocks.API{} @@ -137,22 +121,10 @@ func TestNodeResource(t *testing.T) { assert.Error(t, err) workloads := []*types.Workload{ { - ResourceMeta: types.ResourceMeta{ - MemoryRequest: 1, - MemoryLimit: 1, - CPU: types.CPUMap{"0": 100, "1": 30}, - CPUQuotaRequest: 1.3, - CPUQuotaLimit: 1.3, - }, + ResourceArgs: map[string]types.WorkloadResourceArgs{}, }, { - ResourceMeta: types.ResourceMeta{ - MemoryRequest: 2, - MemoryLimit: 2, - CPU: types.CPUMap{"1": 50}, - CPUQuotaRequest: 0.5, - CPUQuotaLimit: 0.5, - }, + ResourceArgs: map[string]types.WorkloadResourceArgs{}, }, } store.On("ListNodeWorkloads", mock.Anything, mock.Anything, mock.Anything).Return(workloads, nil) @@ -166,143 +138,23 @@ func TestNodeResource(t *testing.T) { assert.Contains(t, details, "inspect failed") } -func TestAllocResource(t *testing.T) { - c := NewTestCluster() - scheduler.InitSchedulerV1(c.scheduler) - ctx := context.Background() - podname := "testpod" - opts := &types.DeployOptions{ - Podname: podname, - Entrypoint: &types.Entrypoint{ - Name: "entry", - }, - } - config := types.Config{ - LockTimeout: time.Duration(time.Second * 3), - } - c.config = config - n1 := "n2" - n2 := "n2" - nodeMap := map[string]*types.Node{ - n1: { - NodeMeta: types.NodeMeta{ - Name: n1, - Labels: map[string]string{"test": "1"}, - CPU: types.CPUMap{"0": 100}, - MemCap: 100, - }, - Available: false, - }, - n2: { - NodeMeta: types.NodeMeta{ - Name: n2, - CPU: types.CPUMap{"0": 100}, - MemCap: 100, - }, - Available: true, - }, - } - - store := c.store.(*storemocks.Store) - defer store.AssertExpectations(t) - - // Defines for below. - opts.NodeFilter.Includes = []string{n2} - - // define scheduleInfos - scheduleInfos := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: n2, - CPU: types.CPUMap{"0": 100}, - MemCap: 100, - }, - Capacity: 10, // can deploy 10 - }, - } - - sched := c.scheduler.(*schedulermocks.Scheduler) - defer sched.AssertExpectations(t) - total := 3 - sched.On("SelectStorageNodes", mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos, total, nil) - sched.On("SelectVolumeNodes", mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos, nil, total, nil) - sched.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, 0, types.ErrInsufficientMEM).Once() - testAllocFailedAsInsufficientMemory(t, c, opts, nodeMap) - - sched.On("SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(scheduleInfos, total, nil) - testAllocFailedAsMakeDeployStatusError(t, c, opts, nodeMap) - store.On("MakeDeployStatus", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) - - testAllocFailedAsWrongDeployMethod(t, c, opts, nodeMap) - testAllocFailedAsCommonDivisionError(t, c, opts, nodeMap) - - // Mocks for all. - opts.DeployStrategy = strategy.Fill - oldFillFunc := strategy.Plans[strategy.Fill] - strategy.Plans[strategy.Fill] = func(ctx context.Context, sis []strategy.Info, need, _, limit int) (map[string]int, error) { - dis := make(map[string]int) - for _, si := range sis { - dis[si.Nodename] = 3 - } - return dis, nil - } - defer func() { - strategy.Plans[strategy.Fill] = oldFillFunc - }() - - // success - opts.ResourceOpts = types.ResourceOptions{CPUQuotaLimit: 1, MemoryLimit: 1, StorageLimit: 1} - opts.Count = 1 - _, _, err := c.doAllocResource(ctx, nodeMap, opts) - assert.NoError(t, err) -} - -func testAllocFailedAsMakeDeployStatusError(t *testing.T, c *Calcium, opts *types.DeployOptions, nodeMap map[string]*types.Node) { - store := c.store.(*storemocks.Store) - store.On("MakeDeployStatus", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(types.ErrNoETCD).Once() - _, _, err := c.doAllocResource(context.Background(), nodeMap, opts) - assert.Error(t, err) -} - -func testAllocFailedAsInsufficientMemory(t *testing.T, c *Calcium, opts *types.DeployOptions, nodeMap map[string]*types.Node) { - opts.ResourceOpts = types.ResourceOptions{CPUQuotaLimit: 1, MemoryLimit: 1} - _, _, err := c.doAllocResource(context.Background(), nodeMap, opts) - assert.Error(t, err) -} - -func testAllocFailedAsWrongDeployMethod(t *testing.T, c *Calcium, opts *types.DeployOptions, nodeMap map[string]*types.Node) { - - opts.DeployStrategy = "invalid" - _, _, err := c.doAllocResource(context.Background(), nodeMap, opts) - assert.Error(t, err) - opts.DeployStrategy = "AUTO" -} - -func testAllocFailedAsCommonDivisionError(t *testing.T, c *Calcium, opts *types.DeployOptions, nodeMap map[string]*types.Node) { - opts.DeployStrategy = strategy.Auto - old := strategy.Plans[strategy.Auto] - strategy.Plans[strategy.Auto] = func(ctx context.Context, _ []strategy.Info, need, total, _ int) (map[string]int, error) { - return nil, types.ErrInsufficientRes - } - defer func() { - strategy.Plans[strategy.Auto] = old - }() - - _, _, err := c.doAllocResource(context.Background(), nodeMap, opts) - assert.Error(t, err) -} - func TestRemapResource(t *testing.T) { c := NewTestCluster() store := &storemocks.Store{} c.store = store + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) + plugin.On("GetRemapArgs", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetRemapArgsResponse{ + EngineArgsMap: map[string]types.EngineArgs{}, + }, nil) engine := &enginemocks.API{} node := &types.Node{Engine: engine} workload := &types.Workload{ - ResourceMeta: types.ResourceMeta{ - CPUQuotaLimit: 1, - }, + ResourceArgs: map[string]types.WorkloadResourceArgs{}, } store.On("ListNodeWorkloads", mock.Anything, mock.Anything, mock.Anything).Return([]*types.Workload{workload}, nil) ch := make(chan enginetypes.VirtualizationRemapMessage, 1) diff --git a/cluster/calcium/wal_test.go b/cluster/calcium/wal_test.go index 45f5dafc0..d20dc039e 100644 --- a/cluster/calcium/wal_test.go +++ b/cluster/calcium/wal_test.go @@ -9,6 +9,8 @@ import ( enginemocks "github.com/projecteru2/core/engine/mocks" enginetypes "github.com/projecteru2/core/engine/types" lockmocks "github.com/projecteru2/core/lock/mocks" + "github.com/projecteru2/core/resources" + resourcemocks "github.com/projecteru2/core/resources/mocks" storemocks "github.com/projecteru2/core/store/mocks" "github.com/projecteru2/core/types" @@ -46,6 +48,11 @@ func TestHandleCreateWorkloadError(t *testing.T) { wal, err := newWAL(c.config, c) require.NoError(t, err) c.wal = wal + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) engine := &enginemocks.API{} node := &types.Node{ @@ -93,6 +100,11 @@ func TestHandleCreateWorkloadHandled(t *testing.T) { wal, err := newWAL(c.config, c) require.NoError(t, err) c.wal = wal + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) node := &types.Node{ NodeMeta: types.NodeMeta{Name: "nodename"}, @@ -133,6 +145,18 @@ func TestHandleCreateLambda(t *testing.T) { wal, err := newWAL(c.config, c) require.NoError(t, err) c.wal = wal + plugin := c.resource.GetPlugins()[0].(*resourcemocks.Plugin) + plugin.On("GetNodeResourceInfo", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetNodeResourceInfoResponse{ + ResourceInfo: &resources.NodeResourceInfo{}, + Diffs: []string{"hhh"}, + }, nil) + plugin.On("SetNodeResourceUsage", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&resources.SetNodeResourceUsageResponse{ + Before: types.NodeResourceArgs{}, + After: types.NodeResourceArgs{}, + }, nil) + plugin.On("GetRemapArgs", mock.Anything, mock.Anything, mock.Anything).Return(&resources.GetRemapArgsResponse{ + EngineArgsMap: map[string]types.EngineArgs{}, + }, nil) _, err = c.wal.Log(eventCreateLambda, "workloadid") require.NoError(t, err) @@ -165,16 +189,12 @@ func TestHandleCreateLambda(t *testing.T) { eng.On("VirtualizationRemove", mock.Anything, wrk.ID, true, true). Return(nil). Once() - eng.On("VirtualizationResourceRemap", mock.Anything, mock.Anything).Return(nil, nil).Once() store.On("GetWorkloads", mock.Anything, []string{wrk.ID}). Return([]*types.Workload{wrk}, nil). Twice() store.On("RemoveWorkload", mock.Anything, wrk). Return(nil). Once() - store.On("UpdateNodeResource", mock.Anything, node, mock.Anything, mock.Anything). - Return(nil). - Once() store.On("ListNodeWorkloads", mock.Anything, mock.Anything, mock.Anything).Return(nil, nil).Once() lock := &lockmocks.DistributedLock{} lock.On("Lock", mock.Anything).Return(context.TODO(), nil) diff --git a/cluster/cluster.go b/cluster/cluster.go index 8ace051e8..3a051a2f8 100644 --- a/cluster/cluster.go +++ b/cluster/cluster.go @@ -63,6 +63,7 @@ type Cluster interface { NodeStatusStream(ctx context.Context) chan *types.NodeStatus // node resource NodeResource(ctx context.Context, nodename string, fix bool) (*types.NodeResource, error) + SendNodeMetrics(ctx context.Context, nodeName string) // calculate capacity CalculateCapacity(context.Context, *types.DeployOptions) (*types.CapacityMessage, error) // meta workloads diff --git a/cluster/mocks/Cluster.go b/cluster/mocks/Cluster.go index 2b33782a9..c00c6c884 100644 --- a/cluster/mocks/Cluster.go +++ b/cluster/mocks/Cluster.go @@ -845,6 +845,11 @@ func (_m *Cluster) Send(ctx context.Context, opts *types.SendOptions) (chan *typ return r0, r1 } +// SendNodeMetrics provides a mock function with given fields: ctx, nodeName +func (_m *Cluster) SendNodeMetrics(ctx context.Context, nodeName string) { + _m.Called(ctx, nodeName) +} + // SetNode provides a mock function with given fields: ctx, opts func (_m *Cluster) SetNode(ctx context.Context, opts *types.SetNodeOptions) (*types.Node, error) { ret := _m.Called(ctx, opts) diff --git a/core.go b/core.go index a91c134ef..89983235b 100644 --- a/core.go +++ b/core.go @@ -11,6 +11,8 @@ import ( "testing" "time" + "github.com/prometheus/client_golang/prometheus/promhttp" + "github.com/projecteru2/core/auth" "github.com/projecteru2/core/cluster/calcium" "github.com/projecteru2/core/engine/factory" @@ -23,7 +25,6 @@ import ( "github.com/projecteru2/core/version" "github.com/getsentry/sentry-go" - "github.com/prometheus/client_golang/prometheus/promhttp" cli "github.com/urfave/cli/v2" _ "go.uber.org/automaxprocs" "google.golang.org/grpc" @@ -68,11 +69,6 @@ func serve(c *cli.Context) error { defer sentryDefer() } - if err := metrics.InitMetrics(config); err != nil { - log.Errorf(nil, "[main] %v", err) //nolint - return err - } - // init engine cache and start engine cache checker factory.InitEngineCache(c.Context, config) diff --git a/engine/docker/container.go b/engine/docker/container.go index c3104ee20..5d4dc02d8 100644 --- a/engine/docker/container.go +++ b/engine/docker/container.go @@ -14,12 +14,7 @@ import ( "strings" "time" - corecluster "github.com/projecteru2/core/cluster" - enginetypes "github.com/projecteru2/core/engine/types" - "github.com/projecteru2/core/log" - "github.com/projecteru2/core/types" - coretypes "github.com/projecteru2/core/types" - "github.com/projecteru2/core/utils" + "github.com/projecteru2/core/engine" dockertypes "github.com/docker/docker/api/types" dockercontainer "github.com/docker/docker/api/types/container" @@ -27,7 +22,10 @@ import ( dockerslice "github.com/docker/docker/api/types/strslice" "github.com/docker/go-connections/nat" "github.com/docker/go-units" - "github.com/pkg/errors" + + enginetypes "github.com/projecteru2/core/engine/types" + "github.com/projecteru2/core/log" + coretypes "github.com/projecteru2/core/types" ) const ( @@ -82,6 +80,15 @@ func loadRawArgs(b []byte) (*RawArgs, error) { // VirtualizationCreate create a workload func (e *Engine) VirtualizationCreate(ctx context.Context, opts *enginetypes.VirtualizationCreateOptions) (*enginetypes.VirtualizationCreated, error) { // nolint r := &enginetypes.VirtualizationCreated{} + var err error + + // parse engine args to resource options + opts.VirtualizationResource, err = engine.MakeVirtualizationResource(opts.EngineArgs) + if err != nil { + log.Errorf(ctx, "[VirtualizationCreate] failed to parse engine args %+v, err %v", opts.EngineArgs, err) + return r, coretypes.ErrInvalidEngineArgs + } + // memory should more than 4MiB if opts.Memory > 0 && opts.Memory < minMemory || opts.Memory < 0 { return r, coretypes.ErrBadMemory @@ -117,19 +124,17 @@ func (e *Engine) VirtualizationCreate(ctx context.Context, opts *enginetypes.Vir if networkMode == "" { networkMode = dockercontainer.NetworkMode(e.config.Docker.NetworkMode) } - // log config, copy to avoid concurrent writes - logConfig := map[string]string{} - for k, v := range opts.LogConfig { - logConfig[k] = v + // log config + if opts.LogConfig == nil { + opts.LogConfig = map[string]string{} } - - logConfig["mode"] = "non-blocking" - logConfig["max-buffer-size"] = "4m" - logConfig["tag"] = fmt.Sprintf("%s {{.ID}}", opts.Name) + opts.LogConfig["mode"] = "non-blocking" + opts.LogConfig["max-buffer-size"] = "4m" + opts.LogConfig["tag"] = fmt.Sprintf("%s {{.ID}}", opts.Name) if opts.Debug { opts.LogType = e.config.Docker.Log.Type for k, v := range e.config.Docker.Log.Config { - logConfig[k] = v + opts.LogConfig[k] = v } } // add node IP @@ -164,7 +169,7 @@ func (e *Engine) VirtualizationCreate(ctx context.Context, opts *enginetypes.Vir return r, err } - resource := makeResourceSetting(opts.Quota, opts.Memory, opts.CPU, opts.NUMANode) + resource := makeResourceSetting(opts.Quota, opts.Memory, opts.CPU, opts.NUMANode, false) // set ulimits if len(rArgs.Ulimits) == 0 { resource.Ulimits = []*units.Ulimit{ @@ -207,7 +212,7 @@ func (e *Engine) VirtualizationCreate(ctx context.Context, opts *enginetypes.Vir DNS: opts.DNS, LogConfig: dockercontainer.LogConfig{ Type: opts.LogType, - Config: logConfig, + Config: opts.LogConfig, }, NetworkMode: networkMode, RestartPolicy: dockercontainer.RestartPolicy{ @@ -267,61 +272,6 @@ func (e *Engine) VirtualizationCreate(ctx context.Context, opts *enginetypes.Vir return r, err } -// VirtualizationResourceRemap to re-distribute resource according to the whole picture -// supposedly it's exclusively executed, so free feel to operate IO from remote dockerd -func (e *Engine) VirtualizationResourceRemap(ctx context.Context, opts *enginetypes.VirtualizationRemapOptions) (<-chan enginetypes.VirtualizationRemapMessage, error) { - // calculate share pool - sharePool := []string{} - for cpuID, available := range opts.CPUAvailable { - if available >= opts.CPUShareBase { - sharePool = append(sharePool, cpuID) - } - } - shareCPUSet := strings.Join(sharePool, ",") - if shareCPUSet == "" { - info, err := e.Info(ctx) - if err != nil { - return nil, errors.WithStack(err) - } - shareCPUSet = fmt.Sprintf("0-%d", info.NCPU-1) - } - - // filter out workloads non-binding - freeWorkloadResources := map[string]enginetypes.VirtualizationResource{} - for workloadID, resource := range opts.WorkloadResources { - if resource.CPU == nil { - freeWorkloadResources[workloadID] = resource - } - } - - // update! - ch := make(chan enginetypes.VirtualizationRemapMessage) - pool := utils.NewGoroutinePool(10) - go func() { - defer close(ch) - for id, resource := range freeWorkloadResources { - pool.Go(ctx, func(id string, resource enginetypes.VirtualizationResource) func() { - return func() { - updateConfig := dockercontainer.UpdateConfig{Resources: dockercontainer.Resources{ - CPUQuota: int64(resource.Quota * float64(corecluster.CPUPeriodBase)), - CPUPeriod: corecluster.CPUPeriodBase, - CpusetCpus: shareCPUSet, - CPUShares: defaultCPUShare, - }} - _, err := e.client.ContainerUpdate(ctx, id, updateConfig) - ch <- enginetypes.VirtualizationRemapMessage{ - ID: id, - Error: err, - } - } - }(id, resource)) - } - pool.Wait(ctx) - }() - - return ch, nil -} - // VirtualizationCopyTo copy things to virtualization func (e *Engine) VirtualizationCopyTo(ctx context.Context, ID, target string, content []byte, uid, gid int, mode int64) error { return withTarfileDump(ctx, target, content, uid, gid, mode, func(target, tarfile string) error { @@ -349,14 +299,8 @@ func (e *Engine) VirtualizationStop(ctx context.Context, ID string, gracefulTime } // VirtualizationRemove remove virtualization -func (e *Engine) VirtualizationRemove(ctx context.Context, ID string, removeVolumes, force bool) (err error) { - if err = e.client.ContainerRemove(ctx, ID, dockertypes.ContainerRemoveOptions{RemoveVolumes: removeVolumes, Force: force}); err == nil { - return nil - } - if strings.Contains(err.Error(), "No such container") { - return types.ErrWorkloadNotExists - } - return +func (e *Engine) VirtualizationRemove(ctx context.Context, ID string, removeVolumes, force bool) error { + return e.client.ContainerRemove(ctx, ID, dockertypes.ContainerRemoveOptions{RemoveVolumes: removeVolumes, Force: force}) } // VirtualizationInspect get virtualization info @@ -458,22 +402,29 @@ func (e *Engine) VirtualizationWait(ctx context.Context, ID, state string) (*eng // VirtualizationUpdateResource update virtualization resource func (e *Engine) VirtualizationUpdateResource(ctx context.Context, ID string, opts *enginetypes.VirtualizationResource) error { - if opts.Memory > 0 && opts.Memory < minMemory || opts.Memory < 0 { + // parse engine args to resource options + resourceOpts, err := engine.MakeVirtualizationResource(opts.EngineArgs) + if err != nil { + log.Errorf(ctx, "[VirtualizationUpdateResource] failed to parse engine args %+v, workload id %v, err %v", opts.EngineArgs, ID, err) + return coretypes.ErrInvalidEngineArgs + } + + if resourceOpts.Memory > 0 && resourceOpts.Memory < minMemory || resourceOpts.Memory < 0 { return coretypes.ErrBadMemory } - if opts.VolumeChanged { - log.Errorf(ctx, "[VirtualizationUpdateResource] docker engine not support rebinding volume resource: %v", opts.Volumes) + if len(opts.Volumes) > 0 || resourceOpts.VolumeChanged { + log.Errorf(ctx, "[VirtualizationUpdateResource] docker engine not support rebinding volume resource: %v", resourceOpts.Volumes) return coretypes.ErrNotSupport } - memory := opts.Memory + memory := resourceOpts.Memory if memory == 0 { memory = maxMemory } - quota := opts.Quota - cpuMap := opts.CPU - numaNode := opts.NUMANode + quota := resourceOpts.Quota + cpuMap := resourceOpts.CPU + numaNode := resourceOpts.NUMANode // unlimited cpu if quota == 0 || len(cpuMap) == 0 { info, err := e.Info(ctx) // TODO can fixed in docker engine, support empty Cpusetcpus, or use cache to speed up @@ -490,9 +441,9 @@ func (e *Engine) VirtualizationUpdateResource(ctx context.Context, ID string, op } } - newResource := makeResourceSetting(quota, memory, cpuMap, numaNode) + newResource := makeResourceSetting(quota, memory, cpuMap, numaNode, resourceOpts.Remap) updateConfig := dockercontainer.UpdateConfig{Resources: newResource} - _, err := e.client.ContainerUpdate(ctx, ID, updateConfig) + _, err = e.client.ContainerUpdate(ctx, ID, updateConfig) return err } diff --git a/engine/docker/helper.go b/engine/docker/helper.go index eda709c38..23361eb95 100644 --- a/engine/docker/helper.go +++ b/engine/docker/helper.go @@ -100,7 +100,7 @@ func makeMountPaths(opts *enginetypes.VirtualizationCreateOptions) ([]string, ma return binds, volumes } -func makeResourceSetting(cpu float64, memory int64, cpuMap map[string]int64, numaNode string) dockercontainer.Resources { +func makeResourceSetting(cpu float64, memory int64, cpuMap map[string]int64, numaNode string, remap bool) dockercontainer.Resources { resource := dockercontainer.Resources{} resource.CPUQuota = 0 @@ -120,11 +120,16 @@ func makeResourceSetting(cpu float64, memory int64, cpuMap map[string]int64, num resource.CpusetCpus = strings.Join(cpuIDs, ",") // numaNode will empty or numaNode resource.CpusetMems = numaNode - // unrestrain cpu quota for binding - resource.CPUQuota = -1 - // cpu share for fragile pieces - if _, divpart := math.Modf(cpu); divpart > 0 { - resource.CPUShares = int64(math.Round(float64(1024) * divpart)) + + if remap { + resource.CPUShares = int64(1024) + } else { + // unrestrained cpu quota for binding + resource.CPUQuota = -1 + // cpu share for fragile pieces + if _, divpart := math.Modf(cpu); divpart > 0 { + resource.CPUShares = int64(math.Round(float64(1024) * divpart)) + } } } resource.Memory = memory diff --git a/engine/engine.go b/engine/engine.go index a034fc6c0..e4ac8299a 100644 --- a/engine/engine.go +++ b/engine/engine.go @@ -38,7 +38,6 @@ type API interface { BuildContent(ctx context.Context, scm coresource.Source, opts *enginetypes.BuildContentOptions) (string, io.Reader, error) VirtualizationCreate(ctx context.Context, opts *enginetypes.VirtualizationCreateOptions) (*enginetypes.VirtualizationCreated, error) - VirtualizationResourceRemap(context.Context, *enginetypes.VirtualizationRemapOptions) (<-chan enginetypes.VirtualizationRemapMessage, error) VirtualizationCopyTo(ctx context.Context, ID, target string, content []byte, uid, gid int, mode int64) error VirtualizationStart(ctx context.Context, ID string) error VirtualizationStop(ctx context.Context, ID string, gracefulTimeout time.Duration) error diff --git a/engine/mocks/API.go b/engine/mocks/API.go index 908b18692..266b5ce0b 100644 --- a/engine/mocks/API.go +++ b/engine/mocks/API.go @@ -695,29 +695,6 @@ func (_m *API) VirtualizationResize(ctx context.Context, ID string, height uint, return r0 } -// VirtualizationResourceRemap provides a mock function with given fields: _a0, _a1 -func (_m *API) VirtualizationResourceRemap(_a0 context.Context, _a1 *types.VirtualizationRemapOptions) (<-chan types.VirtualizationRemapMessage, error) { - ret := _m.Called(_a0, _a1) - - var r0 <-chan types.VirtualizationRemapMessage - if rf, ok := ret.Get(0).(func(context.Context, *types.VirtualizationRemapOptions) <-chan types.VirtualizationRemapMessage); ok { - r0 = rf(_a0, _a1) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(<-chan types.VirtualizationRemapMessage) - } - } - - var r1 error - if rf, ok := ret.Get(1).(func(context.Context, *types.VirtualizationRemapOptions) error); ok { - r1 = rf(_a0, _a1) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // VirtualizationStart provides a mock function with given fields: ctx, ID func (_m *API) VirtualizationStart(ctx context.Context, ID string) error { ret := _m.Called(ctx, ID) diff --git a/engine/transform.go b/engine/transform.go new file mode 100644 index 000000000..5c67327d5 --- /dev/null +++ b/engine/transform.go @@ -0,0 +1,23 @@ +package engine + +import ( + "encoding/json" + + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/engine/types" +) + +func MakeVirtualizationResource(engineArgs map[string]interface{}) (types.VirtualizationResource, error) { + var res types.VirtualizationResource + body, err := json.Marshal(engineArgs) + if err != nil { + return res, err + } + if err = json.Unmarshal(body, &res); err != nil { + logrus.Errorf("[MakeVirtualizationResource] failed to unmarshal from engine args %v, err: %v", string(body), err) + return res, err + } + res.EngineArgs = engineArgs + return res, nil +} diff --git a/engine/types/virtualization.go b/engine/types/virtualization.go index a48070ecc..ee6dde2d5 100644 --- a/engine/types/virtualization.go +++ b/engine/types/virtualization.go @@ -2,14 +2,16 @@ package types // VirtualizationResource define resources type VirtualizationResource struct { - CPU map[string]int64 // for cpu binding - Quota float64 // for cpu quota - Memory int64 // for memory binding - Storage int64 - NUMANode string // numa node - Volumes []string - VolumePlan map[string]map[string]int64 // literal VolumePlan - VolumeChanged bool // indicate whether new volumes contained in realloc request + EngineArgs map[string]interface{} `json:"-"` + CPU map[string]int64 `json:"cpu_map"` // for cpu binding + Quota float64 `json:"cpu"` // for cpu quota + Memory int64 `json:"memory"` // for memory binding + Storage int64 `json:"storage"` + NUMANode string `json:"numa_node"` // numa node + Volumes []string `json:"volumes"` + VolumePlan map[string]map[string]int64 `json:"volume_plan"` // literal VolumePlan + VolumeChanged bool `json:"volume_changed"` // indicate whether new volumes contained in realloc request + Remap bool `json:"remap"` } // VirtualizationCreateOptions use for create virtualization target @@ -34,8 +36,6 @@ type VirtualizationCreateOptions struct { Networks map[string]string - Volumes []string - LogType string LogConfig map[string]string diff --git a/engine/virt/virt.go b/engine/virt/virt.go index 43f2add54..af9f7f432 100644 --- a/engine/virt/virt.go +++ b/engine/virt/virt.go @@ -181,7 +181,14 @@ func (v *Virt) BuildContent(ctx context.Context, scm coresource.Source, opts *en } // VirtualizationCreate creates a guest. -func (v *Virt) VirtualizationCreate(ctx context.Context, opts *enginetypes.VirtualizationCreateOptions) (*enginetypes.VirtualizationCreated, error) { +func (v *Virt) VirtualizationCreate(ctx context.Context, opts *enginetypes.VirtualizationCreateOptions) (guest *enginetypes.VirtualizationCreated, err error) { + // parse engine args to resource options + opts.VirtualizationResource, err = engine.MakeVirtualizationResource(opts.EngineArgs) + if err != nil { + log.Errorf(ctx, "[VirtualizationCreate] failed to parse engine args %+v, err %v", opts.EngineArgs, err) + return nil, coretypes.ErrInvalidEngineArgs + } + vols, err := v.parseVolumes(opts.Volumes) if err != nil { return nil, err @@ -213,15 +220,6 @@ func (v *Virt) VirtualizationCreate(ctx context.Context, opts *enginetypes.Virtu }, nil } -// VirtualizationResourceRemap . -func (v *Virt) VirtualizationResourceRemap(ctx context.Context, opts *enginetypes.VirtualizationRemapOptions) (<-chan enginetypes.VirtualizationRemapMessage, error) { - // VM does not support binding cores. - log.Warn(ctx, "virtualizationResourceRemap is not supported by vm") - ch := make(chan enginetypes.VirtualizationRemapMessage) - defer close(ch) - return ch, nil -} - // VirtualizationCopyTo copies one. func (v *Virt) VirtualizationCopyTo(ctx context.Context, ID, dest string, content []byte, uid, gid int, mode int64) error { return v.client.CopyToGuest(ctx, ID, dest, bytes.NewReader(content), true, true) @@ -320,7 +318,14 @@ func (v *Virt) VirtualizationWait(ctx context.Context, ID, state string) (*engin } // VirtualizationUpdateResource updates resource. -func (v *Virt) VirtualizationUpdateResource(ctx context.Context, ID string, opts *enginetypes.VirtualizationResource) error { +func (v *Virt) VirtualizationUpdateResource(ctx context.Context, ID string, resourceOpts *enginetypes.VirtualizationResource) error { + // parse engine args to resource options + opts, err := engine.MakeVirtualizationResource(resourceOpts.EngineArgs) + if err != nil { + log.Errorf(ctx, "[VirtualizationCreate] failed to parse engine args %+v, err %v", opts.EngineArgs, err) + return coretypes.ErrInvalidEngineArgs + } + vols, err := v.parseVolumes(opts.Volumes) if err != nil { return err diff --git a/go.mod b/go.mod index 85d93c9de..78f814c67 100644 --- a/go.mod +++ b/go.mod @@ -4,21 +4,32 @@ go 1.18 require ( github.com/CMGS/statsd v0.0.0-20160223095033-48c421b3c1ab + github.com/Microsoft/hcsshim v0.8.11 // indirect + github.com/alexcesaro/statsd v2.0.0+incompatible // indirect github.com/alicebob/miniredis/v2 v2.14.3 github.com/cenkalti/backoff/v4 v4.0.2 - github.com/cornelk/hashmap v1.0.2-0.20210201213917-c93d96ce6b8a + github.com/containerd/containerd v1.4.12 // indirect + github.com/containerd/continuity v0.0.0-20200710164510-efbc4488d8fe // indirect github.com/docker/distribution v2.8.0+incompatible github.com/docker/docker v20.10.0+incompatible github.com/docker/go-connections v0.4.0 + github.com/docker/go-metrics v0.0.1 // indirect github.com/docker/go-units v0.4.0 + github.com/docker/libtrust v0.0.0-20160708172513-aabc10ec26b7 // indirect github.com/getsentry/sentry-go v0.9.0 github.com/go-git/go-git/v5 v5.2.0 github.com/go-ping/ping v0.0.0-20210407214646-e4e642a95741 github.com/go-redis/redis/v8 v8.8.2 github.com/google/uuid v1.1.2 + github.com/gorilla/mux v1.7.4 // indirect github.com/jinzhu/configor v1.2.0 + github.com/moby/sys/mount v0.2.0 // indirect + github.com/moby/term v0.0.0-20201110203204-bea5bbe245bf // indirect + github.com/morikuni/aec v1.0.0 // indirect github.com/muroq/redislock v0.0.0-20210327061935-5425e33e6f9f + github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.0.2 + github.com/opencontainers/runc v1.0.3 // indirect github.com/patrickmn/go-cache v2.1.0+incompatible github.com/pkg/errors v0.9.1 github.com/projecteru2/libyavirt v0.0.0-20220330115351-ec63a4a270d3 @@ -32,35 +43,33 @@ require ( go.etcd.io/etcd/client/pkg/v3 v3.5.0 go.etcd.io/etcd/client/v3 v3.5.0 go.etcd.io/etcd/tests/v3 v3.5.0 + go.opencensus.io v0.22.1 // indirect go.uber.org/automaxprocs v1.3.0 golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 golang.org/x/exp v0.0.0-20220328175248-053ad81199eb golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f golang.org/x/sync v0.0.0-20210220032951-036812b2e83c + google.golang.org/appengine v1.6.3 // indirect google.golang.org/grpc v1.40.0 google.golang.org/protobuf v1.27.1 ) +require github.com/cornelk/hashmap v1.0.1 + require ( github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78 // indirect github.com/BurntSushi/toml v0.3.1 // indirect github.com/Microsoft/go-winio v0.4.16-0.20201130162521-d1ffc52c7331 // indirect - github.com/Microsoft/hcsshim v0.8.11 // indirect - github.com/alexcesaro/statsd v2.0.0+incompatible // indirect github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.1.1 // indirect github.com/containerd/cgroups v0.0.0-20200531161412-0dbf7f05ba59 // indirect - github.com/containerd/containerd v1.4.13 // indirect - github.com/containerd/continuity v0.0.0-20200710164510-efbc4488d8fe // indirect github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd/v22 v22.3.2 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/dchest/siphash v1.1.0 // indirect github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f // indirect - github.com/docker/go-metrics v0.0.1 // indirect - github.com/docker/libtrust v0.0.0-20160708172513-aabc10ec26b7 // indirect github.com/dustin/go-humanize v1.0.0 // indirect github.com/emirpasic/gods v1.12.0 // indirect github.com/form3tech-oss/jwt-go v3.2.3+incompatible // indirect @@ -70,7 +79,6 @@ require ( github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/protobuf v1.5.2 // indirect github.com/google/btree v1.0.1 // indirect - github.com/gorilla/mux v1.7.4 // indirect github.com/gorilla/websocket v1.4.2 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect @@ -82,14 +90,9 @@ require ( github.com/kevinburke/ssh_config v0.0.0-20190725054713-01f96b0aa0cd // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect github.com/mitchellh/go-homedir v1.1.0 // indirect - github.com/moby/sys/mount v0.2.0 // indirect github.com/moby/sys/mountinfo v0.4.1 // indirect - github.com/moby/term v0.0.0-20201110203204-bea5bbe245bf // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.1 // indirect - github.com/morikuni/aec v1.0.0 // indirect - github.com/opencontainers/go-digest v1.0.0 // indirect - github.com/opencontainers/runc v1.0.3 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/common v0.26.0 // indirect @@ -108,7 +111,6 @@ require ( go.etcd.io/etcd/pkg/v3 v3.5.0 // indirect go.etcd.io/etcd/raft/v3 v3.5.0 // indirect go.etcd.io/etcd/server/v3 v3.5.0 // indirect - go.opencensus.io v0.22.1 // indirect go.opentelemetry.io/contrib v0.20.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0 // indirect go.opentelemetry.io/otel v0.20.0 // indirect @@ -129,7 +131,6 @@ require ( golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba // indirect golang.org/x/tools v0.1.8-0.20211029000441-d6a9af8af023 // indirect golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 // indirect - google.golang.org/appengine v1.6.3 // indirect google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect gopkg.in/warnings.v0 v0.1.2 // indirect diff --git a/go.sum b/go.sum index ef5f8d531..401ca86d6 100644 --- a/go.sum +++ b/go.sum @@ -94,8 +94,8 @@ github.com/containerd/cgroups v0.0.0-20200531161412-0dbf7f05ba59/go.mod h1:pA0z1 github.com/containerd/console v0.0.0-20180822173158-c12b1e7919c1/go.mod h1:Tj/on1eG8kiEhd0+fhSDzsPAFESxzBBvdyEgyryXffw= github.com/containerd/console v1.0.2/go.mod h1:ytZPjGgY2oeTkAONYafi2kSj0aYggsf8acV1PGKCbzQ= github.com/containerd/containerd v1.3.2/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= -github.com/containerd/containerd v1.4.13 h1:Z0CbagVdn9VN4K6htOCY/jApSw8YKP+RdLZ5dkXF8PM= -github.com/containerd/containerd v1.4.13/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= +github.com/containerd/containerd v1.4.12 h1:V+SHzYmhng/iju6M5nFrpTTusrhidoxKTwdwLw+u4c4= +github.com/containerd/containerd v1.4.12/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= github.com/containerd/continuity v0.0.0-20190426062206-aaeac12a7ffc/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y= github.com/containerd/continuity v0.0.0-20200710164510-efbc4488d8fe h1:PEmIrUvwG9Yyv+0WKZqjXfSFDeZjs/q15g0m08BYS9k= github.com/containerd/continuity v0.0.0-20200710164510-efbc4488d8fe/go.mod h1:cECdGN1O8G9bgKTlLhuPJimka6Xb/Gg7vYzCTNVxhvo= @@ -115,8 +115,8 @@ github.com/coreos/go-systemd/v22 v22.0.0/go.mod h1:xO0FLkIi5MaZafQlIrOotqXZ90ih+ github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/cornelk/hashmap v1.0.2-0.20210201213917-c93d96ce6b8a h1:qv8c3h9mYAfJP3xPuMlhm12taC8VI0Vq4a9mivqCZBc= -github.com/cornelk/hashmap v1.0.2-0.20210201213917-c93d96ce6b8a/go.mod h1:SI48x/mQnWtjsJuM7GgmODo4o5O8FhGJgClCRmtOtIQ= +github.com/cornelk/hashmap v1.0.1 h1:RXGcy29hEdLLV8T6aK4s+BAd4tq4+3Hq50N2GoG0uIg= +github.com/cornelk/hashmap v1.0.1/go.mod h1:8wbysTUDnwJGrPZ1Iwsou3m+An6sldFrJItjRhfegCw= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0 h1:EoUDS0afbrsXAZ9YQ9jdu/mZ2sXgT1/2yyNng4PGlyM= @@ -663,8 +663,6 @@ golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxT golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= golang.org/x/exp v0.0.0-20200908183739-ae8ad444f925/go.mod h1:1phAWC201xIgDyaFpmDeZkgf70Q4Pd/CNqfRtVPtxNw= -golang.org/x/exp v0.0.0-20220323204016-c86f0da35e87 h1:pFVxvJFSIGjuRLaw0mTTDfxn/AMeSYzY6Y/Hr7adkVU= -golang.org/x/exp v0.0.0-20220323204016-c86f0da35e87/go.mod h1:lgLbSvA5ygNOMpwM/9anMpWVlVJ7Z+cHWq/eFuinpGE= golang.org/x/exp v0.0.0-20220328175248-053ad81199eb h1:pC9Okm6BVmxEw76PUu0XUbOTQ92JX11hfvqTjAV3qxM= golang.org/x/exp v0.0.0-20220328175248-053ad81199eb/go.mod h1:lgLbSvA5ygNOMpwM/9anMpWVlVJ7Z+cHWq/eFuinpGE= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= diff --git a/metrics/handler.go b/metrics/handler.go index 81789efd4..433b11b37 100644 --- a/metrics/handler.go +++ b/metrics/handler.go @@ -13,14 +13,14 @@ import ( func (m *Metrics) ResourceMiddleware(cluster cluster.Cluster) func(http.Handler) http.Handler { return func(h http.Handler) http.Handler { return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - ctx, cancel := context.WithTimeout(context.TODO(), m.Config.GlobalTimeout) + ctx, cancel := context.WithTimeout(r.Context(), m.Config.GlobalTimeout) defer cancel() nodes, err := cluster.ListPodNodes(ctx, &types.ListNodesOptions{All: true}) if err != nil { log.Errorf(ctx, "[ResourceMiddleware] Get all nodes err %v", err) } for node := range nodes { - m.SendNodeInfo(node.Metrics()) + cluster.SendNodeMetrics(ctx, node.Name) } h.ServeHTTP(w, r) }) diff --git a/metrics/metrics.go b/metrics/metrics.go index c808203a4..39408041f 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -1,25 +1,25 @@ package metrics import ( - "fmt" "os" + "strconv" + "sync" "github.com/projecteru2/core/log" + "github.com/projecteru2/core/resources" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" statsdlib "github.com/CMGS/statsd" "github.com/prometheus/client_golang/prometheus" + "golang.org/x/exp/maps" ) const ( - cpuMap = "core.node.%s.cpu.%s" - memStats = "core.node.%s.memory" - storageStats = "core.node.%s.storage" - memUsedStats = "core.node.%s.memory.used" - storageUsedStats = "core.node.%s.storage.used" - cpuUsedStats = "core.node.%s.cpu.used" - deployCount = "core.%s.deploy.count" + deployCountKey = "core.%s.deploy.count" + deployCountName = "core_deploy" + gaugeType = "gauge" + counterType = "counter" ) // Metrics define metrics @@ -30,85 +30,7 @@ type Metrics struct { Hostname string statsdClient *statsdlib.Client - MemoryCapacity *prometheus.GaugeVec - MemoryUsed *prometheus.GaugeVec - StorageCapacity *prometheus.GaugeVec - StorageUsed *prometheus.GaugeVec - CPUMap *prometheus.GaugeVec - CPUUsed *prometheus.GaugeVec - DeployCount *prometheus.CounterVec -} - -// SendNodeInfo update node resource capacity -func (m *Metrics) SendNodeInfo(nm *types.NodeMetrics) { - nodename := nm.Name - podname := nm.Podname - memory := nm.Memory - memoryUsed := nm.MemoryUsed - storage := nm.Storage - storageUsed := nm.StorageUsed - cpuUsed := nm.CPUUsed - - if m.MemoryCapacity != nil { - m.MemoryCapacity.WithLabelValues(podname, nodename).Set(memory) - } - - if m.MemoryUsed != nil { - m.MemoryUsed.WithLabelValues(podname, nodename).Set(memoryUsed) - } - - if m.StorageCapacity != nil { - m.StorageCapacity.WithLabelValues(podname, nodename).Set(storage) - } - - if m.StorageUsed != nil { - m.StorageUsed.WithLabelValues(podname, nodename).Set(storageUsed) - } - - if m.CPUUsed != nil { - m.CPUUsed.WithLabelValues(podname, nodename).Set(cpuUsed) - } - - cleanedNodeName := utils.CleanStatsdMetrics(nodename) - for cpuid, value := range nm.CPU { - val := float64(value) - - if m.CPUMap != nil { - m.CPUMap.WithLabelValues(podname, nodename, cpuid).Set(val) - } - - if m.StatsdAddr == "" { - continue - } - - if err := m.gauge(fmt.Sprintf(cpuMap, cleanedNodeName, cpuid), val); err != nil { - log.Errorf(nil, "[SendNodeInfo] Error occurred while sending cpu data to statsd: %v", err) //nolint - } - } - - if m.StatsdAddr == "" { - return - } - - if err := m.gauge(fmt.Sprintf(memStats, cleanedNodeName), memory); err != nil { - log.Errorf(nil, "[SendNodeInfo] Error occurred while sending memory data to statsd: %v", err) //nolint - } - - if err := m.gauge(fmt.Sprintf(storageStats, cleanedNodeName), storage); err != nil { - log.Errorf(nil, "[SendNodeInfo] Error occurred while sending storage data to statsd: %v", err) //nolint - } - - if err := m.gauge(fmt.Sprintf(memUsedStats, cleanedNodeName), memoryUsed); err != nil { - log.Errorf(nil, "[SendNodeInfo] Error occurred while sending memory used data to statsd: %v", err) //nolint - } - - if err := m.gauge(fmt.Sprintf(storageUsedStats, cleanedNodeName), storageUsed); err != nil { - log.Errorf(nil, "[SendNodeInfo] Error occurred while sending storage used data to statsd: %v", err) //nolint - } - - if err := m.gauge(fmt.Sprintf(cpuUsedStats, cleanedNodeName), cpuUsed); err != nil { - log.Errorf(nil, "[SendNodeInfo] Error occurred while sending cpu used data to statsd: %v", err) //nolint - } + Collectors map[string]prometheus.Collector } // Lazy connect @@ -129,6 +51,9 @@ func (m *Metrics) checkConn() error { } func (m *Metrics) gauge(key string, value float64) error { + if m.StatsdAddr == "" { + return nil + } if err := m.checkConn(); err != nil { return err } @@ -137,6 +62,9 @@ func (m *Metrics) gauge(key string, value float64) error { } func (m *Metrics) count(key string, n int, rate float32) error { + if m.StatsdAddr == "" { + return nil + } if err := m.checkConn(); err != nil { return err } @@ -147,24 +75,55 @@ func (m *Metrics) count(key string, n int, rate float32) error { // SendDeployCount update deploy counter func (m *Metrics) SendDeployCount(n int) { log.Info("[Metrics] Update deploy counter") - if m.DeployCount != nil { - m.DeployCount.WithLabelValues(m.Hostname).Add(float64(n)) + metrics := &resources.Metrics{ + Name: deployCountName, + Labels: []string{m.Hostname}, + Key: deployCountKey, + Value: strconv.Itoa(n), } - if m.StatsdAddr == "" { - return - } - key := fmt.Sprintf(deployCount, m.Hostname) - if err := m.count(key, n, 1.0); err != nil { - log.Errorf(nil, "[SendDeployCount] Error occurred while counting: %v", err) //nolint + m.SendMetrics(metrics) +} + +// SendMetrics update metrics +func (m *Metrics) SendMetrics(metrics ...*resources.Metrics) { + for _, metric := range metrics { + collector, ok := m.Collectors[metric.Name] + if !ok { + log.Errorf(nil, "[SendMetrics] Collector not found: %s", metric.Name) //nolint + continue + } + switch collector.(type) { // nolint: gocritic + case *prometheus.GaugeVec: + value, err := strconv.ParseFloat(metric.Value, 64) + if err != nil { + log.Errorf(nil, "[SendMetrics] Error occurred while parsing %v value %v: %v", metric.Name, metric.Value, err) //nolint + } + collector.(*prometheus.GaugeVec).WithLabelValues(metric.Labels...).Set(value) + if err := m.gauge(metric.Key, value); err != nil { + log.Errorf(nil, "[SendMetrics] Error occurred while sending %v data to statsd: %v", metric.Name, err) //nolint + } + case *prometheus.CounterVec: + value, err := strconv.ParseInt(metric.Value, 10, 32) + if err != nil { + log.Errorf(nil, "[SendMetrics] Error occurred while parsing %v value %v: %v", metric.Name, metric.Value, err) //nolint + } + collector.(*prometheus.CounterVec).WithLabelValues(metric.Labels...).Add(float64(value)) + if err := m.count(metric.Key, int(value), 1.0); err != nil { + log.Errorf(nil, "[SendMetrics] Error occurred while sending %v data to statsd: %v", metric.Name, err) //nolint + } + default: + log.Errorf(nil, "[SendMetrics] Unknown collector type: %T", collector) //nolint + } } } // Client is a metrics obj var Client = Metrics{} +var once sync.Once // InitMetrics new a metrics obj -func InitMetrics(config types.Config) error { +func InitMetrics(config types.Config, metricsDescriptions []*resources.MetricsDescription) error { hostname, err := os.Hostname() if err != nil { return err @@ -173,47 +132,28 @@ func InitMetrics(config types.Config) error { Config: config, StatsdAddr: config.Statsd, Hostname: utils.CleanStatsdMetrics(hostname), + Collectors: map[string]prometheus.Collector{}, + } + + for _, desc := range metricsDescriptions { + switch desc.Type { + case gaugeType: + collector := prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Name: desc.Name, + Help: desc.Help, + }, desc.Labels) + Client.Collectors[desc.Name] = collector + case counterType: + collector := prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: desc.Name, + Help: desc.Help, + }, desc.Labels) + Client.Collectors[desc.Name] = collector + } } - Client.MemoryCapacity = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Name: "memory_capacity", - Help: "node available memory.", - }, []string{"podname", "nodename"}) - - Client.MemoryUsed = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Name: "memory_used", - Help: "node used memory.", - }, []string{"podname", "nodename"}) - - Client.StorageCapacity = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Name: "storage_capacity", - Help: "node available storage.", - }, []string{"podname", "nodename"}) - - Client.StorageUsed = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Name: "storage_used", - Help: "node used storage.", - }, []string{"podname", "nodename"}) - - Client.CPUMap = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Name: "cpu_map", - Help: "node available cpu.", - }, []string{"podname", "nodename", "cpuid"}) - - Client.CPUUsed = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Name: "cpu_used", - Help: "node used cpu.", - }, []string{"podname", "nodename"}) - - Client.DeployCount = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: "core_deploy", - Help: "core deploy counter", - }, []string{"hostname"}) - - prometheus.MustRegister( - Client.DeployCount, Client.MemoryCapacity, - Client.StorageCapacity, Client.CPUMap, - Client.MemoryUsed, Client.StorageUsed, Client.CPUUsed, - ) + once.Do(func() { + prometheus.MustRegister(maps.Values(Client.Collectors)...) + }) return nil } diff --git a/resources/binary.go b/resources/binary.go new file mode 100644 index 000000000..0193b5811 --- /dev/null +++ b/resources/binary.go @@ -0,0 +1,296 @@ +package resources + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "os/exec" + "path" + "reflect" + "strings" + + enginetypes "github.com/projecteru2/core/engine/types" + "github.com/projecteru2/core/log" + coretypes "github.com/projecteru2/core/types" +) + +// BinaryPlugin . +type BinaryPlugin struct { + path string + config coretypes.Config +} + +func (bp *BinaryPlugin) getArgs(req interface{}) []string { + t := reflect.TypeOf(req) + if t.Kind() != reflect.Struct { + return nil + } + v := reflect.ValueOf(req) + args := []string{} + + for i := 0; i < t.NumField(); i++ { + fieldType := t.Field(i).Type + fieldValue := v.Field(i).Interface() + jsonTag := t.Field(i).Tag.Get("json") + + switch { + case fieldType.Kind() == reflect.Map: + if v.Field(i).IsZero() { + break + } + body, err := json.Marshal(fieldValue) + if err != nil { + break + } + args = append(args, "--"+jsonTag, string(body)) + case fieldType.Kind() == reflect.Slice: + for j := 0; j < v.Field(i).Len(); j++ { + if v.Field(i).Index(j).Kind() == reflect.Map { + body, err := json.Marshal(v.Field(i).Index(j).Interface()) + if err != nil { + break + } + args = append(args, "--"+jsonTag, string(body)) + } else { + args = append(args, "--"+jsonTag, fmt.Sprintf("%v", v.Field(i).Index(j).Interface())) + } + } + case fieldType.Kind() == reflect.Bool: + if fieldValue.(bool) { + args = append(args, "--"+jsonTag) + } + default: + args = append(args, "--"+jsonTag, fmt.Sprintf("%v", fieldValue)) + } + } + return args +} + +func (bp *BinaryPlugin) execCommand(cmd *exec.Cmd) (output, log string, err error) { + var stdout, stderr bytes.Buffer + cmd.Stdout = &stdout + cmd.Stderr = &stderr + + err = cmd.Run() + output = stdout.String() + log = stderr.String() + if err != nil { + err = fmt.Errorf("err: %v, output: %v, log: %v", err, output, log) + } + return output, log, err +} + +// calls the plugin and gets json response +func (bp *BinaryPlugin) call(ctx context.Context, cmd string, req interface{}, resp interface{}) error { + ctx, cancel := context.WithTimeout(ctx, bp.config.ResourcePluginsTimeout) + defer cancel() + + args := bp.getArgs(req) + args = append([]string{cmd}, args...) + command := exec.CommandContext(ctx, bp.path, args...) // nolint: gosec + command.Dir = bp.config.ResourcePluginsDir + log.Infof(ctx, "[callBinaryPlugin] command: %s %s", bp.path, strings.Join(args, " ")) + pluginOutput, pluginLog, err := bp.execCommand(command) + + defer log.Infof(ctx, "[callBinaryPlugin] log from plugin %s: %s", bp.path, pluginLog) + defer log.Infof(ctx, "[callBinaryPlugin] output from plugin %s: %s", bp.path, pluginOutput) + + if err != nil { + log.Errorf(ctx, "[callBinaryPlugin] failed to run plugin %s, command %v, err %s", bp.path, args, err) + return err + } + + if len(pluginOutput) == 0 { + pluginOutput = "{}" + } + if err := json.Unmarshal([]byte(pluginOutput), resp); err != nil { + log.Errorf(ctx, "[callBinaryPlugin] failed to unmarshal output of plugin %s, command %v, output %s, err %s", bp.path, args, pluginOutput, err) + return err + } + return nil +} + +// GetNodesDeployCapacity . +func (bp *BinaryPlugin) GetNodesDeployCapacity(ctx context.Context, nodes []string, resourceOpts coretypes.WorkloadResourceOpts) (resp *GetNodesDeployCapacityResponse, err error) { + req := GetNodesDeployCapacityRequest{ + NodeNames: nodes, + ResourceOpts: resourceOpts, + } + resp = &GetNodesDeployCapacityResponse{} + err = bp.call(ctx, getNodesCapacityCommand, req, resp) + return resp, err +} + +func (bp *BinaryPlugin) getNodeResourceInfo(ctx context.Context, nodeName string, workloads []*coretypes.Workload, fix bool) (resp *GetNodeResourceInfoResponse, err error) { + workloadMap := map[string]coretypes.WorkloadResourceArgs{} + for _, workload := range workloads { + workloadMap[workload.ID] = workload.ResourceArgs[bp.Name()] + } + + req := GetNodeResourceInfoRequest{ + NodeName: nodeName, + WorkloadMap: workloadMap, + Fix: fix, + } + resp = &GetNodeResourceInfoResponse{} + if err = bp.call(ctx, getNodeResourceInfoCommand, req, resp); err != nil { + return nil, err + } + return resp, nil +} + +// GetNodeResourceInfo . +func (bp *BinaryPlugin) GetNodeResourceInfo(ctx context.Context, nodeName string, workloads []*coretypes.Workload) (resp *GetNodeResourceInfoResponse, err error) { + return bp.getNodeResourceInfo(ctx, nodeName, workloads, false) +} + +// FixNodeResource . +func (bp *BinaryPlugin) FixNodeResource(ctx context.Context, nodeName string, workloads []*coretypes.Workload) (resp *GetNodeResourceInfoResponse, err error) { + return bp.getNodeResourceInfo(ctx, nodeName, workloads, true) +} + +// SetNodeResourceInfo . +func (bp *BinaryPlugin) SetNodeResourceInfo(ctx context.Context, nodeName string, resourceCapacity coretypes.NodeResourceArgs, resourceUsage coretypes.NodeResourceArgs) (*SetNodeResourceInfoResponse, error) { + req := SetNodeResourceInfoRequest{ + NodeName: nodeName, + Capacity: resourceCapacity, + Usage: resourceUsage, + } + resp := &SetNodeResourceInfoResponse{} + return resp, bp.call(ctx, setNodeResourceInfoCommand, req, resp) +} + +// GetDeployArgs . +func (bp *BinaryPlugin) GetDeployArgs(ctx context.Context, nodeName string, deployCount int, resourceOpts coretypes.WorkloadResourceOpts) (resp *GetDeployArgsResponse, err error) { + req := GetDeployArgsRequest{ + NodeName: nodeName, + DeployCount: deployCount, + ResourceOpts: resourceOpts, + } + resp = &GetDeployArgsResponse{} + if err := bp.call(ctx, getDeployArgsCommand, req, resp); err != nil { + return nil, err + } + return resp, nil +} + +// GetReallocArgs . +func (bp *BinaryPlugin) GetReallocArgs(ctx context.Context, nodeName string, originResourceArgs coretypes.WorkloadResourceArgs, resourceOpts coretypes.WorkloadResourceOpts) (resp *GetReallocArgsResponse, err error) { + req := GetReallocArgsRequest{ + NodeName: nodeName, + Old: originResourceArgs, + ResourceOpts: resourceOpts, + } + resp = &GetReallocArgsResponse{} + if err := bp.call(ctx, getReallocArgsCommand, req, resp); err != nil { + return nil, err + } + return resp, nil +} + +// GetRemapArgs . +func (bp *BinaryPlugin) GetRemapArgs(ctx context.Context, nodeName string, workloadMap map[string]*coretypes.Workload) (*GetRemapArgsResponse, error) { + workloadResourceArgsMap := map[string]coretypes.WorkloadResourceArgs{} + for workloadID, workload := range workloadMap { + workloadResourceArgsMap[workloadID] = workload.ResourceArgs[bp.Name()] + } + + req := GetRemapArgsRequest{ + NodeName: nodeName, + WorkloadMap: workloadResourceArgsMap, + } + resp := &GetRemapArgsResponse{} + if err := bp.call(ctx, getRemapArgsCommand, req, resp); err != nil { + return nil, err + } + return resp, nil +} + +func (bp *BinaryPlugin) SetNodeResourceUsage(ctx context.Context, nodeName string, nodeResourceOpts coretypes.NodeResourceOpts, nodeResourceArgs coretypes.NodeResourceArgs, workloadResourceArgs []coretypes.WorkloadResourceArgs, delta bool, incr bool) (*SetNodeResourceUsageResponse, error) { + req := SetNodeResourceUsageRequest{ + NodeName: nodeName, + WorkloadResourceArgs: workloadResourceArgs, + NodeResourceOpts: nodeResourceOpts, + NodeResourceArgs: nodeResourceArgs, + Delta: delta, + Decr: !incr, + } + + resp := &SetNodeResourceUsageResponse{} + if err := bp.call(ctx, setNodeResourceUsageCommand, req, resp); err != nil { + return nil, err + } + return resp, nil +} + +func (bp *BinaryPlugin) SetNodeResourceCapacity(ctx context.Context, nodeName string, nodeResourceOpts coretypes.NodeResourceOpts, nodeResourceArgs coretypes.NodeResourceArgs, delta bool, incr bool) (*SetNodeResourceCapacityResponse, error) { + req := SetNodeResourceCapacityRequest{ + NodeName: nodeName, + NodeResourceOpts: nodeResourceOpts, + NodeResourceArgs: nodeResourceArgs, + Delta: delta, + Decr: !incr, + } + + resp := &SetNodeResourceCapacityResponse{} + if err := bp.call(ctx, setNodeResourceCapacityCommand, req, resp); err != nil { + return nil, err + } + return resp, nil +} + +// AddNode . +func (bp *BinaryPlugin) AddNode(ctx context.Context, nodeName string, resourceOpts coretypes.NodeResourceOpts, nodeInfo *enginetypes.Info) (resp *AddNodeResponse, err error) { + req := AddNodeRequest{ + NodeName: nodeName, + ResourceOpts: resourceOpts, + } + resp = &AddNodeResponse{} + if err := bp.call(ctx, addNodeCommand, req, resp); err != nil { + return nil, err + } + return resp, nil +} + +// RemoveNode . +func (bp *BinaryPlugin) RemoveNode(ctx context.Context, nodeName string) (*RemoveNodeResponse, error) { + req := RemoveNodeRequest{ + NodeName: nodeName, + } + resp := &RemoveNodeResponse{} + return resp, bp.call(ctx, removeNodeCommand, req, resp) +} + +// GetMostIdleNode . +func (bp *BinaryPlugin) GetMostIdleNode(ctx context.Context, nodeNames []string) (*GetMostIdleNodeResponse, error) { + req := GetMostIdleNodeRequest{ + NodeNames: nodeNames, + } + resp := &GetMostIdleNodeResponse{} + return resp, bp.call(ctx, getMostIdleNodeCommand, req, resp) +} + +// Name . +func (bp *BinaryPlugin) Name() string { + return path.Base(bp.path) +} + +// GetMetricsDescription . +func (bp *BinaryPlugin) GetMetricsDescription(ctx context.Context) (*GetMetricsDescriptionResponse, error) { + req := GetMetricsDescriptionRequest{} + resp := &GetMetricsDescriptionResponse{} + return resp, bp.call(ctx, getMetricsDescriptionCommand, req, resp) +} + +// ResolveNodeResourceInfoToMetrics . +func (bp *BinaryPlugin) ResolveNodeResourceInfoToMetrics(ctx context.Context, podName string, nodeName string, nodeResourceInfo *NodeResourceInfo) (*ResolveNodeResourceInfoToMetricsResponse, error) { + req := ResolveNodeResourceInfoToMetricsRequest{ + PodName: podName, + NodeName: nodeName, + Capacity: nodeResourceInfo.Capacity, + Usage: nodeResourceInfo.Usage, + } + resp := &ResolveNodeResourceInfoToMetricsResponse{} + return resp, bp.call(ctx, resolveNodeResourceInfoToMetricsCommand, req, resp) +} diff --git a/resources/cpumem/cpumem.go b/resources/cpumem/cpumem.go index e36f3ece7..acf82c98f 100644 --- a/resources/cpumem/cpumem.go +++ b/resources/cpumem/cpumem.go @@ -2,180 +2,336 @@ package cpumem import ( "context" + "strconv" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - "github.com/projecteru2/core/types" - - "github.com/pkg/errors" + enginetypes "github.com/projecteru2/core/engine/types" + "github.com/projecteru2/core/resources" + "github.com/projecteru2/core/resources/cpumem/models" + "github.com/projecteru2/core/resources/cpumem/types" + coretypes "github.com/projecteru2/core/types" ) -type cpuMemRequest struct { - CPUQuotaRequest float64 - CPUQuotaLimit float64 - CPUBind bool - CPU types.CPUMap - - memoryRequest int64 - memoryLimit int64 +// Plugin wrapper of CPUMem +type Plugin struct { + c *models.CPUMem } -// 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, - CPU: opts.CPU, +// NewPlugin creates a new Plugin +func NewPlugin(config coretypes.Config) (*Plugin, error) { + c, err := models.NewCPUMem(config) + if err != nil { + return nil, err } - return cmr, cmr.Validate() + return &Plugin{c: c}, nil } -// Type . -func (cm cpuMemRequest) Type() types.ResourceType { - t := types.ResourceCPU | types.ResourceMemory - if cm.CPUBind { - t |= types.ResourceCPUBind +// GetDeployArgs . +func (c *Plugin) GetDeployArgs(ctx context.Context, nodeName string, deployCount int, resourceOpts coretypes.WorkloadResourceOpts) (*resources.GetDeployArgsResponse, error) { + workloadResourceOpts := &types.WorkloadResourceOpts{} + if err := workloadResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err } - return t + engineArgs, resourceArgs, err := c.c.GetDeployArgs(ctx, nodeName, deployCount, workloadResourceOpts) + if err != nil { + return nil, err + } + + resp := &resources.GetDeployArgsResponse{} + err = resources.ToResp(map[string]interface{}{ + "engine_args": engineArgs, + "resource_args": resourceArgs, + }, resp) + return resp, err } -// Validate . -func (cm *cpuMemRequest) Validate() error { - if cm.CPUQuotaRequest == 0 && cm.CPUQuotaLimit > 0 { - cm.CPUQuotaRequest = cm.CPUQuotaLimit +// GetReallocArgs . +func (c *Plugin) GetReallocArgs(ctx context.Context, nodeName string, originResourceArgs coretypes.WorkloadResourceArgs, resourceOpts coretypes.WorkloadResourceOpts) (*resources.GetReallocArgsResponse, error) { + workloadResourceOpts := &types.WorkloadResourceOpts{} + if err := workloadResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err } - if cm.memoryLimit < 0 || cm.memoryRequest < 0 { - return errors.Wrap(types.ErrBadMemory, "limit or request less than 0") + originWorkloadResourceArgs := &types.WorkloadResourceArgs{} + if err := originWorkloadResourceArgs.ParseFromRawParams(coretypes.RawParams(originResourceArgs)); err != nil { + return nil, err } - if cm.CPUQuotaLimit < 0 || cm.CPUQuotaRequest < 0 { - return errors.Wrap(types.ErrBadCPU, "limit or request less than 0") + + engineArgs, delta, resourceArgs, err := c.c.GetReallocArgs(ctx, nodeName, originWorkloadResourceArgs, workloadResourceOpts) + if err != nil { + return nil, err } - if cm.CPUQuotaRequest == 0 && cm.CPUBind { - return errors.Wrap(types.ErrBadCPU, "unlimited request with bind") + + resp := &resources.GetReallocArgsResponse{} + err = resources.ToResp(map[string]interface{}{ + "engine_args": engineArgs, + "delta": delta, + "resource_args": resourceArgs, + }, resp) + return resp, err +} + +// GetRemapArgs . +func (c *Plugin) GetRemapArgs(ctx context.Context, nodeName string, workloadMap map[string]*coretypes.Workload) (*resources.GetRemapArgsResponse, error) { + workloadResourceArgsMap, err := c.workloadMapToWorkloadResourceArgsMap(workloadMap) + if err != nil { + return nil, err } - if cm.memoryRequest == 0 && cm.memoryLimit > 0 { - cm.memoryRequest = cm.memoryLimit + engineArgs, err := c.c.GetRemapArgs(ctx, nodeName, workloadResourceArgsMap) + if err != nil { + return nil, err } - // 如果需求量大于限制量,悄咪咪的把限制量抬到需求量的水平,做成名义上的软限制 - if cm.memoryLimit > 0 && cm.memoryRequest > 0 && cm.memoryRequest > cm.memoryLimit { - cm.memoryLimit = cm.memoryRequest + + resp := &resources.GetRemapArgsResponse{} + err = resources.ToResp(map[string]interface{}{ + "engine_args": engineArgs, + }, resp) + return resp, err +} + +// GetNodesDeployCapacity . +func (c *Plugin) GetNodesDeployCapacity(ctx context.Context, nodeNames []string, resourceOpts coretypes.WorkloadResourceOpts) (*resources.GetNodesDeployCapacityResponse, error) { + workloadResourceOpts := &types.WorkloadResourceOpts{} + if err := workloadResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err } - if cm.CPUQuotaRequest > 0 && cm.CPUQuotaLimit > 0 && cm.CPUQuotaRequest > cm.CPUQuotaLimit { - cm.CPUQuotaLimit = cm.CPUQuotaRequest + + nodesDeployCapacity, total, err := c.c.GetNodesDeployCapacity(ctx, nodeNames, workloadResourceOpts) + if err != nil { + return nil, err } - // if CPUBind=true, set cpu request=limit to solve the dilemma - // only deal with cpu limit>request but not vice versa - if cm.CPUBind && cm.CPUQuotaRequest > 0 && cm.CPUQuotaLimit > 0 && cm.CPUQuotaLimit > cm.CPUQuotaRequest { - cm.CPUQuotaRequest = cm.CPUQuotaLimit + + resp := &resources.GetNodesDeployCapacityResponse{} + err = resources.ToResp(map[string]interface{}{ + "nodes": nodesDeployCapacity, + "total": total, + }, resp) + return resp, err +} + +// GetMostIdleNode . +func (c *Plugin) GetMostIdleNode(ctx context.Context, nodeNames []string) (*resources.GetMostIdleNodeResponse, error) { + nodeName, priority, err := c.c.GetMostIdleNode(ctx, nodeNames) + if err != nil { + return nil, err } - return nil + + resp := &resources.GetMostIdleNodeResponse{} + err = resources.ToResp(map[string]interface{}{ + "node": nodeName, + "priority": priority, + }, resp) + return resp, err +} + +// GetNodeResourceInfo . +func (c *Plugin) GetNodeResourceInfo(ctx context.Context, nodeName string, workloads []*coretypes.Workload) (*resources.GetNodeResourceInfoResponse, error) { + return c.getNodeResourceInfo(ctx, nodeName, workloads, false) +} + +// FixNodeResource . +func (c *Plugin) FixNodeResource(ctx context.Context, nodeName string, workloads []*coretypes.Workload) (*resources.GetNodeResourceInfoResponse, error) { + return c.getNodeResourceInfo(ctx, nodeName, workloads, true) } -// MakeScheduler . -func (cm cpuMemRequest) MakeScheduler() resourcetypes.SchedulerV2 { - return func(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo) (plans resourcetypes.ResourcePlans, total int, err error) { - schedulerV1, err := scheduler.GetSchedulerV1() - if err != nil { - return +// SetNodeResourceUsage . +func (c *Plugin) SetNodeResourceUsage(ctx context.Context, nodeName string, resourceOpts coretypes.NodeResourceOpts, resourceArgs coretypes.NodeResourceArgs, workloadResourceArgs []coretypes.WorkloadResourceArgs, delta bool, incr bool) (*resources.SetNodeResourceUsageResponse, error) { + var nodeResourceOpts *types.NodeResourceOpts + var nodeResourceArgs *types.NodeResourceArgs + var workloadResourceArgsList []*types.WorkloadResourceArgs + + if resourceOpts != nil { + nodeResourceOpts = &types.NodeResourceOpts{} + if err := nodeResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err } + } + + if resourceArgs != nil { + nodeResourceArgs = &types.NodeResourceArgs{} + if err := nodeResourceArgs.ParseFromRawParams(coretypes.RawParams(resourceArgs)); err != nil { + return nil, err + } + } + + if workloadResourceArgs != nil { + workloadResourceArgsList = make([]*types.WorkloadResourceArgs, len(workloadResourceArgs)) + for i, workloadResourceArg := range workloadResourceArgs { + workloadResourceArgsList[i] = &types.WorkloadResourceArgs{} + if err := workloadResourceArgsList[i].ParseFromRawParams(coretypes.RawParams(workloadResourceArg)); err != nil { + return nil, err + } + } + } + + before, after, err := c.c.SetNodeResourceUsage(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, workloadResourceArgsList, delta, incr) + if err != nil { + return nil, err + } - var CPUPlans map[string][]types.CPUMap - switch { - case !cm.CPUBind || cm.CPUQuotaRequest == 0: - scheduleInfos, total, err = schedulerV1.SelectMemoryNodes(ctx, scheduleInfos, cm.CPUQuotaRequest, cm.memoryRequest) - case cm.CPU != nil: - scheduleInfos[0], CPUPlans, total, err = schedulerV1.ReselectCPUNodes(ctx, scheduleInfos[0], cm.CPU, cm.CPUQuotaRequest, cm.memoryRequest) - default: - scheduleInfos, CPUPlans, total, err = schedulerV1.SelectCPUNodes(ctx, scheduleInfos, cm.CPUQuotaRequest, cm.memoryRequest) + resp := &resources.SetNodeResourceUsageResponse{} + err = resources.ToResp(map[string]interface{}{ + "before": before, + "after": after, + }, resp) + return resp, err +} + +// SetNodeResourceCapacity . +func (c *Plugin) SetNodeResourceCapacity(ctx context.Context, nodeName string, resourceOpts coretypes.NodeResourceOpts, resourceArgs coretypes.NodeResourceArgs, delta bool, incr bool) (*resources.SetNodeResourceCapacityResponse, error) { + var nodeResourceOpts *types.NodeResourceOpts + var nodeResourceArgs *types.NodeResourceArgs + + if resourceOpts != nil { + nodeResourceOpts = &types.NodeResourceOpts{} + if err := nodeResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err + } + } + if resourceArgs != nil { + nodeResourceArgs = &types.NodeResourceArgs{} + if err := nodeResourceArgs.ParseFromRawParams(coretypes.RawParams(resourceArgs)); err != nil { + return nil, err } - return ResourcePlans{ - memoryRequest: cm.memoryRequest, - memoryLimit: cm.memoryLimit, - CPUQuotaRequest: cm.CPUQuotaRequest, - CPUQuotaLimit: cm.CPUQuotaLimit, - CPUPlans: CPUPlans, - capacity: resourcetypes.GetCapacity(scheduleInfos), - }, total, err } + + before, after, err := c.c.SetNodeResourceCapacity(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, delta, incr) + if err != nil { + return nil, err + } + + resp := &resources.SetNodeResourceCapacityResponse{} + err = resources.ToResp(map[string]interface{}{ + "before": before, + "after": after, + }, resp) + return resp, err } -// Rate for global strategy -func (cm cpuMemRequest) Rate(node types.Node) float64 { - if cm.CPUBind { - return cm.CPUQuotaRequest / float64(len(node.InitCPU)) +// SetNodeResourceInfo . +func (c *Plugin) SetNodeResourceInfo(ctx context.Context, nodeName string, resourceCapacity coretypes.NodeResourceArgs, resourceUsage coretypes.NodeResourceArgs) (*resources.SetNodeResourceInfoResponse, error) { + capacity := &types.NodeResourceArgs{} + if err := capacity.ParseFromRawParams(coretypes.RawParams(resourceCapacity)); err != nil { + return nil, err } - return float64(cm.memoryRequest) / float64(node.InitMemCap) + + usage := &types.NodeResourceArgs{} + if err := usage.ParseFromRawParams(coretypes.RawParams(resourceUsage)); err != nil { + return nil, err + } + + if err := c.c.SetNodeResourceInfo(ctx, nodeName, capacity, usage); err != nil { + return nil, err + } + return &resources.SetNodeResourceInfoResponse{}, nil } -// ResourcePlans . -type ResourcePlans struct { - memoryRequest int64 - memoryLimit int64 +// AddNode . +func (c *Plugin) AddNode(ctx context.Context, nodeName string, resourceOpts coretypes.NodeResourceOpts, nodeInfo *enginetypes.Info) (*resources.AddNodeResponse, error) { + nodeResourceOpts := &types.NodeResourceOpts{} + if err := nodeResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err + } + + // set default value + if nodeInfo != nil { + if len(nodeResourceOpts.CPUMap) == 0 { + nodeResourceOpts.CPUMap = types.CPUMap{} + for i := 0; i < nodeInfo.NCPU; i++ { + nodeResourceOpts.CPUMap[strconv.Itoa(i)] = c.c.Config.Scheduler.ShareBase + } + } + + if nodeResourceOpts.Memory == 0 { + nodeResourceOpts.Memory = nodeInfo.MemTotal * 8 / 10 // use 80% of real memory + } + } - CPUQuotaRequest float64 - CPUQuotaLimit float64 - CPUPlans map[string][]types.CPUMap + nodeResourceInfo, err := c.c.AddNode(ctx, nodeName, nodeResourceOpts) + if err != nil { + return nil, err + } - capacity map[string]int + resp := &resources.AddNodeResponse{} + err = resources.ToResp(map[string]interface{}{ + "capacity": nodeResourceInfo.Capacity, + "usage": nodeResourceInfo.Usage, + }, resp) + return resp, err } -// Type . -func (rp ResourcePlans) Type() (resourceType types.ResourceType) { - resourceType = types.ResourceCPU | types.ResourceMemory - if rp.CPUPlans != nil { - resourceType |= types.ResourceCPUBind +// RemoveNode . +func (c *Plugin) RemoveNode(ctx context.Context, nodeName string) (*resources.RemoveNodeResponse, error) { + if err := c.c.RemoveNode(ctx, nodeName); err != nil { + return nil, err } - return resourceType + return &resources.RemoveNodeResponse{}, nil } -// Capacity . -func (rp ResourcePlans) Capacity() map[string]int { - return rp.capacity +// Name . +func (c *Plugin) Name() string { + return "cpumem" } -// ApplyChangesOnNode . -func (rp ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { - if rp.CPUPlans != nil { - for _, idx := range indices { - node.CPU.Sub(rp.CPUPlans[node.Name][idx]) +func (c *Plugin) workloadMapToWorkloadResourceArgsMap(workloadMap map[string]*coretypes.Workload) (*types.WorkloadResourceArgsMap, error) { + workloadResourceArgsMap := types.WorkloadResourceArgsMap{} + for workloadID, workload := range workloadMap { + workloadResourceArgs := &types.WorkloadResourceArgs{} + if err := workloadResourceArgs.ParseFromRawParams(coretypes.RawParams(workload.ResourceArgs[c.Name()])); err != nil { + return nil, err } + workloadResourceArgsMap[workloadID] = workloadResourceArgs } - node.MemCap -= rp.memoryRequest * int64(len(indices)) - node.SetCPUUsed(rp.CPUQuotaRequest*float64(len(indices)), types.IncrUsage) + + return &workloadResourceArgsMap, nil } -// RollbackChangesOnNode . -func (rp ResourcePlans) RollbackChangesOnNode(node *types.Node, indices ...int) { - if rp.CPUPlans != nil { - for _, idx := range indices { - node.CPU.Add(rp.CPUPlans[node.Name][idx]) - } +func (c *Plugin) workloadListToWorkloadResourceArgsMap(workloads []*coretypes.Workload) (*types.WorkloadResourceArgsMap, error) { + workloadMap := map[string]*coretypes.Workload{} + for _, workload := range workloads { + workloadMap[workload.ID] = workload } - node.MemCap += rp.memoryRequest * int64(len(indices)) - node.SetCPUUsed(rp.CPUQuotaRequest*float64(len(indices)), types.DecrUsage) + + return c.workloadMapToWorkloadResourceArgsMap(workloadMap) } -// Dispense . -func (rp ResourcePlans) Dispense(opts resourcetypes.DispenseOptions, r *types.ResourceMeta) (*types.ResourceMeta, error) { - if rp.capacity[opts.Node.Name] <= opts.Index { - return nil, errors.WithStack(types.ErrInsufficientCap) +func (c *Plugin) getNodeResourceInfo(ctx context.Context, nodeName string, workloads []*coretypes.Workload, fix bool) (*resources.GetNodeResourceInfoResponse, error) { + workloadResourceArgsMap, err := c.workloadListToWorkloadResourceArgsMap(workloads) + if err != nil { + return nil, err } - r.CPUQuotaLimit = rp.CPUQuotaLimit - r.CPUQuotaRequest = rp.CPUQuotaRequest - r.MemoryLimit = rp.memoryLimit - r.MemoryRequest = rp.memoryRequest - if len(rp.CPUPlans) > 0 { - if p, ok := rp.CPUPlans[opts.Node.Name]; !ok || len(p) <= opts.Index { - return nil, errors.WithStack(types.ErrInsufficientCPU) - } + nodeResourceInfo, diffs, err := c.c.GetNodeResourceInfo(ctx, nodeName, workloadResourceArgsMap, fix) + if err != nil { + return nil, err + } + + resp := &resources.GetNodeResourceInfoResponse{} + err = resources.ToResp(map[string]interface{}{ + "resource_info": nodeResourceInfo, + "diffs": diffs, + }, resp) + return resp, err +} - r.CPU = rp.CPUPlans[opts.Node.Name][opts.Index] - r.NUMANode = opts.Node.GetNUMANode(r.CPU) +// GetMetricsDescription . +func (c *Plugin) GetMetricsDescription(ctx context.Context) (*resources.GetMetricsDescriptionResponse, error) { + resp := &resources.GetMetricsDescriptionResponse{} + err := resources.ToResp(c.c.GetMetricsDescription(), resp) + return resp, err +} + +// ResolveNodeResourceInfoToMetrics . +func (c *Plugin) ResolveNodeResourceInfoToMetrics(ctx context.Context, podName string, nodeName string, info *resources.NodeResourceInfo) (*resources.ResolveNodeResourceInfoToMetricsResponse, error) { + capacity, usage := &types.NodeResourceArgs{}, &types.NodeResourceArgs{} + if err := capacity.ParseFromRawParams(coretypes.RawParams(info.Capacity)); err != nil { + return nil, err + } + if err := usage.ParseFromRawParams(coretypes.RawParams(info.Usage)); err != nil { + return nil, err } - return r, nil + + metrics := c.c.ResolveNodeResourceInfoToMetrics(podName, nodeName, capacity, usage) + resp := &resources.ResolveNodeResourceInfoToMetricsResponse{} + err := resources.ToResp(metrics, resp) + return resp, err } diff --git a/resources/cpumem/cpumem_test.go b/resources/cpumem/cpumem_test.go deleted file mode 100644 index 8c4475783..000000000 --- a/resources/cpumem/cpumem_test.go +++ /dev/null @@ -1,309 +0,0 @@ -package cpumem - -import ( - "context" - "testing" - - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - schedulerMocks "github.com/projecteru2/core/scheduler/mocks" - "github.com/projecteru2/core/types" - - "github.com/pkg/errors" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" -) - -func TestMakeRequest(t *testing.T) { - // Mem request below zero shall fail - _, err := MakeRequest(types.ResourceOptions{ - MemoryRequest: -1, - MemoryLimit: -1, - }) - assert.NotNil(t, err) - - // Mem and cpu request equal to zero will not fail - _, err = MakeRequest(types.ResourceOptions{ - MemoryRequest: 0, - MemoryLimit: 1, - CPUQuotaRequest: 0, - CPUQuotaLimit: 1, - }) - assert.Nil(t, err) - - // Request more then limited will not fail - _, err = MakeRequest(types.ResourceOptions{ - MemoryRequest: 2, - MemoryLimit: 1, - CPUQuotaRequest: 2, - CPUQuotaLimit: 1, - }) - assert.Nil(t, err) - - // Request below zero will fail - _, err = MakeRequest(types.ResourceOptions{ - CPUQuotaRequest: -0.5, - CPUQuotaLimit: -1, - }) - assert.NotNil(t, err) - - // Request unlimited cpu but with cpu bind will fail - _, err = MakeRequest(types.ResourceOptions{ - CPUQuotaRequest: 0, - CPUBind: true, - }) - assert.NotNil(t, err) -} - -func TestType(t *testing.T) { - req, err := MakeRequest(types.ResourceOptions{ - MemoryRequest: 0, - MemoryLimit: 1, - CPUQuotaRequest: 0, - CPUQuotaLimit: 1, - }) - assert.Nil(t, err) - assert.True(t, req.Type()&(types.ResourceCPU|types.ResourceMemory) > 0) - - req, err = MakeRequest(types.ResourceOptions{ - CPUQuotaRequest: 1, - CPUQuotaLimit: 1, - CPUBind: true, - }) - assert.Nil(t, err) - assert.True(t, req.Type()&types.ResourceCPUBind > 0) -} - -func TestRate(t *testing.T) { - req, err := MakeRequest(types.ResourceOptions{ - MemoryRequest: 0, - MemoryLimit: 1, - }) - assert.Nil(t, err) - node := types.Node{ - NodeMeta: types.NodeMeta{ - InitCPU: types.CPUMap{"1": 100, "2": 100}, - InitMemCap: 100, - }, - } - assert.Equal(t, req.Rate(node), 0.01) - req, err = MakeRequest(types.ResourceOptions{ - CPUQuotaRequest: 0, - CPUQuotaLimit: 2, - CPUBind: true, - }) - assert.Nil(t, err) - assert.Equal(t, req.Rate(node), 1.0) -} - -func TestRequestCpuNode(t *testing.T) { - run(t, newRequestCPUNodeTest()) -} - -func TestRequestMemNode(t *testing.T) { - run(t, newRequestMemNodeTest(types.ResourceOptions{ - CPUQuotaRequest: 0.5, - CPUQuotaLimit: 1, - CPUBind: false, - MemoryRequest: 512, - MemoryLimit: 1024, - })) - run(t, newRequestMemNodeTest(types.ResourceOptions{ - CPUQuotaRequest: 0, - CPUQuotaLimit: 0, - CPUBind: false, - MemoryRequest: 512, - MemoryLimit: 1024, - })) -} - -type nodeSchdulerTest interface { - getScheduleInfo() []resourcetypes.ScheduleInfo - getScheduler() scheduler.Scheduler - getRequestOptions() types.ResourceOptions - getNode() *types.Node - assertAfterChanges(t *testing.T) - assertAfterRollback(t *testing.T) -} - -func run(t *testing.T, test nodeSchdulerTest) { - resourceRequest, err := MakeRequest(test.getRequestOptions()) - assert.NoError(t, err) - _, _, err = resourceRequest.MakeScheduler()(context.TODO(), []resourcetypes.ScheduleInfo{}) - assert.Error(t, err) - - s := test.getScheduler() - prevSche, _ := scheduler.GetSchedulerV1() - scheduler.InitSchedulerV1(s) - defer func() { - scheduler.InitSchedulerV1(prevSche) - }() - - resourceRequest, err = MakeRequest(test.getRequestOptions()) - assert.NoError(t, err) - - sche := resourceRequest.MakeScheduler() - - plans, _, err := sche(context.TODO(), test.getScheduleInfo()) - assert.Nil(t, err) - - var node = test.getNode() - - assert.True(t, plans.Type()&(types.ResourceCPU|types.ResourceMemory) > 0) - assert.NotNil(t, plans.Capacity()) - - plans.ApplyChangesOnNode(node, 0) - test.assertAfterChanges(t) - - plans.RollbackChangesOnNode(node, 0) - test.assertAfterRollback(t) - - opts := resourcetypes.DispenseOptions{ - Node: node, - Index: 0, - } - r := &types.ResourceMeta{} - _, err = plans.Dispense(opts, r) - assert.Nil(t, err) - - opts.Index = 20001 - _, err = plans.Dispense(opts, r) - assert.EqualError(t, err, "cannot alloc a each node plan, not enough capacity") - -} - -type requestCPUNodeTest struct { - node types.Node - scheduleInfos []resourcetypes.ScheduleInfo - cpuMap map[string][]types.CPUMap -} - -func newRequestCPUNodeTest() nodeSchdulerTest { - return &requestCPUNodeTest{ - node: types.Node{ - NodeMeta: types.NodeMeta{ - Name: "TestNode", - CPU: map[string]int64{"0": 10000, "1": 10000}, - NUMA: map[string]string{"0": "0", "1": "1"}, - NUMAMemory: map[string]int64{"0": 1024, "1": 1204}, - MemCap: 10240, - }, - }, - scheduleInfos: []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "TestNode", - CPU: map[string]int64{"0": 10000, "1": 10000}, - NUMA: map[string]string{"0": "0", "1": "1"}, - NUMAMemory: map[string]int64{"0": 1024, "1": 1204}, - MemCap: 10240, - }, - CPUPlan: []types.CPUMap{{"0": 10000, "1": 10000}}, - Capacity: 20000, - }, - }, - cpuMap: map[string][]types.CPUMap{"TestNode": {{"0": 10000, "1": 10000}}}, - } -} - -func (test *requestCPUNodeTest) getScheduleInfo() []resourcetypes.ScheduleInfo { - return test.scheduleInfos -} - -func (test *requestCPUNodeTest) getScheduler() scheduler.Scheduler { - mockScheduler := &schedulerMocks.Scheduler{} - mockScheduler.On( - "SelectCPUNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, - ).Return(test.scheduleInfos, test.cpuMap, 1, nil) - mockScheduler.On( - "SelectMemoryNodess", mock.Anything, mock.Anything, mock.Anything, mock.Anything, - ).Return(test.scheduleInfos, 1, errors.New("should not select memory node here")) - return mockScheduler -} - -func (test *requestCPUNodeTest) getRequestOptions() types.ResourceOptions { - return types.ResourceOptions{ - CPUQuotaRequest: 0.5, - CPUQuotaLimit: 1, - CPUBind: true, - MemoryRequest: 512, - MemoryLimit: 1024, - } -} - -func (test *requestCPUNodeTest) getNode() *types.Node { - return &test.node -} - -func (test *requestCPUNodeTest) assertAfterChanges(t *testing.T) { - assert.Less(t, test.node.CPU["0"], int64(10000)) -} - -func (test *requestCPUNodeTest) assertAfterRollback(t *testing.T) { - assert.Equal(t, test.node.CPU["0"], int64(10000)) -} - -type requestMemNodeTest struct { - node types.Node - scheduleInfos []resourcetypes.ScheduleInfo - reqOpt types.ResourceOptions -} - -func newRequestMemNodeTest(reqOpt types.ResourceOptions) nodeSchdulerTest { - return &requestMemNodeTest{ - node: types.Node{ - NodeMeta: types.NodeMeta{ - Name: "TestNode", - CPU: map[string]int64{"0": 10000, "1": 10000}, - NUMA: map[string]string{"0": "0", "1": "1"}, - NUMAMemory: map[string]int64{"0": 1024, "1": 1204}, - MemCap: 10240, - }, - }, - scheduleInfos: []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "TestNode", - CPU: map[string]int64{"0": 10000, "1": 10000}, - NUMA: map[string]string{"0": "0", "1": "1"}, - NUMAMemory: map[string]int64{"0": 1024, "1": 1204}, - MemCap: 10240, - }, - CPUPlan: []types.CPUMap{{"0": 10000, "1": 10000}}, - Capacity: 20000, - }, - }, - reqOpt: reqOpt, - } -} - -func (test *requestMemNodeTest) getRequestOptions() types.ResourceOptions { - return test.reqOpt -} - -func (test *requestMemNodeTest) getScheduleInfo() []resourcetypes.ScheduleInfo { - return test.scheduleInfos -} - -func (test *requestMemNodeTest) getScheduler() scheduler.Scheduler { - mockScheduler := &schedulerMocks.Scheduler{} - mockScheduler.On( - "SelectCPUNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, - ).Return(test.scheduleInfos, nil, 1, errors.New("should not select memory node here")) - mockScheduler.On( - "SelectMemoryNodes", mock.Anything, mock.Anything, mock.Anything, mock.Anything, - ).Return(test.scheduleInfos, 1, nil) - return mockScheduler -} - -func (test *requestMemNodeTest) getNode() *types.Node { - return &test.node -} - -func (test *requestMemNodeTest) assertAfterChanges(t *testing.T) { - assert.Less(t, test.node.MemCap, int64(10240)) -} - -func (test *requestMemNodeTest) assertAfterRollback(t *testing.T) { - assert.Equal(t, test.node.CPU["0"], int64(10000)) -} diff --git a/resources/cpumem/models/alloc.go b/resources/cpumem/models/alloc.go new file mode 100644 index 000000000..1ebccff2d --- /dev/null +++ b/resources/cpumem/models/alloc.go @@ -0,0 +1,97 @@ +package models + +import ( + "context" + + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/cpumem/schedule" + "github.com/projecteru2/core/resources/cpumem/types" +) + +// GetDeployArgs . +func (c *CPUMem) GetDeployArgs(ctx context.Context, node string, deployCount int, opts *types.WorkloadResourceOpts) ([]*types.EngineArgs, []*types.WorkloadResourceArgs, error) { + if err := opts.Validate(); err != nil { + logrus.Errorf("[GetDeployArgs] invalid resource opts %+v, err: %v", opts, err) + return nil, nil, err + } + + resourceInfo, err := c.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[GetDeployArgs] failed to get resource info of node %v, err: %v", node, err) + return nil, nil, err + } + + if !opts.CPUBind { + return c.doAllocByMemory(resourceInfo, deployCount, opts) + } + + return c.doAllocByCPU(resourceInfo, deployCount, opts) +} + +func (c *CPUMem) doAllocByMemory(resourceInfo *types.NodeResourceInfo, deployCount int, opts *types.WorkloadResourceOpts) ([]*types.EngineArgs, []*types.WorkloadResourceArgs, error) { + if opts.CPURequest > float64(len(resourceInfo.Capacity.CPUMap)) { + return nil, nil, types.ErrInsufficientCPU + } + + availableResourceArgs := resourceInfo.GetAvailableResource() + if opts.MemRequest > 0 && availableResourceArgs.Memory/opts.MemRequest < int64(deployCount) { + return nil, nil, types.ErrInsufficientMem + } + + resEngineArgs := []*types.EngineArgs{} + resResourceArgs := []*types.WorkloadResourceArgs{} + + engineArgs := &types.EngineArgs{ + CPU: opts.CPULimit, + Memory: opts.MemLimit, + } + resourceArgs := &types.WorkloadResourceArgs{ + CPURequest: opts.CPURequest, + CPULimit: opts.CPULimit, + MemoryRequest: opts.MemRequest, + MemoryLimit: opts.MemLimit, + } + + for len(resEngineArgs) < deployCount { + resEngineArgs = append(resEngineArgs, engineArgs) + resResourceArgs = append(resResourceArgs, resourceArgs) + } + return resEngineArgs, resResourceArgs, nil +} + +func (c *CPUMem) doAllocByCPU(resourceInfo *types.NodeResourceInfo, deployCount int, opts *types.WorkloadResourceOpts) ([]*types.EngineArgs, []*types.WorkloadResourceArgs, error) { + cpuPlans := schedule.GetCPUPlans(resourceInfo, nil, c.Config.Scheduler.ShareBase, c.Config.Scheduler.MaxShare, opts) + if len(cpuPlans) < deployCount { + return nil, nil, types.ErrInsufficientResource + } + + cpuPlans = cpuPlans[:deployCount] + resEngineArgs := []*types.EngineArgs{} + resResourceArgs := []*types.WorkloadResourceArgs{} + + for _, cpuPlan := range cpuPlans { + resEngineArgs = append(resEngineArgs, &types.EngineArgs{ + CPU: opts.CPULimit, + CPUMap: cpuPlan.CPUMap, + NUMANode: cpuPlan.NUMANode, + Memory: opts.MemLimit, + }) + + resourceArgs := &types.WorkloadResourceArgs{ + CPURequest: opts.CPURequest, + CPULimit: opts.CPULimit, + MemoryRequest: opts.MemRequest, + MemoryLimit: opts.MemLimit, + CPUMap: cpuPlan.CPUMap, + NUMANode: cpuPlan.NUMANode, + } + if len(resourceArgs.NUMANode) > 0 { + resourceArgs.NUMAMemory = types.NUMAMemory{resourceArgs.NUMANode: resourceArgs.MemoryRequest} + } + + resResourceArgs = append(resResourceArgs, resourceArgs) + } + + return resEngineArgs, resResourceArgs, nil +} diff --git a/resources/cpumem/models/alloc_test.go b/resources/cpumem/models/alloc_test.go new file mode 100644 index 000000000..b8c0f5cd7 --- /dev/null +++ b/resources/cpumem/models/alloc_test.go @@ -0,0 +1,101 @@ +package models + +import ( + "context" + "errors" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/cpumem/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestAlloc(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 1, 2, 4*units.GiB, 100) + node := nodes[0] + + // invalid opts + _, _, err := cpuMem.GetDeployArgs(ctx, node, 1, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: -1, + }) + assert.True(t, errors.Is(err, types.ErrInvalidCPU)) + + // non-existent node + _, _, err = cpuMem.GetDeployArgs(ctx, "xxx", 1, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1, + }) + assert.True(t, errors.Is(err, coretypes.ErrBadCount)) + + // cpu bind + _, _, err = cpuMem.GetDeployArgs(ctx, node, 1, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.1, + }) + assert.Nil(t, err) + + // cpu bind & insufficient resource + _, _, err = cpuMem.GetDeployArgs(ctx, node, 1, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 2.2, + }) + assert.True(t, errors.Is(err, types.ErrInsufficientResource)) + _, _, err = cpuMem.GetDeployArgs(ctx, node, 3, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1, + }) + assert.True(t, errors.Is(err, types.ErrInsufficientResource)) + + // alloc by memory + _, _, err = cpuMem.GetDeployArgs(ctx, node, 1, &types.WorkloadResourceOpts{ + MemRequest: units.GiB, + }) + assert.Nil(t, err) + + // alloc by memory & insufficient cpu + _, _, err = cpuMem.GetDeployArgs(ctx, node, 1, &types.WorkloadResourceOpts{ + MemRequest: units.GiB, + CPURequest: 65535, + }) + assert.True(t, errors.Is(err, types.ErrInsufficientCPU)) + + // alloc by memory & insufficient mem + _, _, err = cpuMem.GetDeployArgs(ctx, node, 1, &types.WorkloadResourceOpts{ + MemRequest: 5 * units.GiB, + CPURequest: 1, + }) + assert.True(t, errors.Is(err, types.ErrInsufficientMem)) + + // mem_request == 0 + _, _, err = cpuMem.GetDeployArgs(ctx, node, 1, &types.WorkloadResourceOpts{ + MemRequest: 0, + CPURequest: 1, + }) + assert.Nil(t, err) + + // numa node + resourceInfo := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + CPU: 4, + CPUMap: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100}, + Memory: 4 * units.GiB, + NUMAMemory: types.NUMAMemory{"0": 2 * units.GiB, "1": 2 * units.GiB}, + NUMA: types.NUMA{"0": "0", "1": "0", "2": "1", "3": "1"}, + }, + } + assert.Nil(t, resourceInfo.Validate()) + assert.Nil(t, cpuMem.doSetNodeResourceInfo(ctx, "numa-node", resourceInfo)) + _, resourceArgs, err := cpuMem.GetDeployArgs(ctx, "numa-node", 2, &types.WorkloadResourceOpts{ + CPUBind: true, + MemRequest: 1 * units.GiB, + CPURequest: 1.3, + }) + assert.Nil(t, err) + assert.ElementsMatch(t, []string{resourceArgs[0].NUMANode, resourceArgs[1].NUMANode}, []string{"0", "1"}) +} diff --git a/resources/cpumem/models/capacity.go b/resources/cpumem/models/capacity.go new file mode 100644 index 000000000..cb0c1e390 --- /dev/null +++ b/resources/cpumem/models/capacity.go @@ -0,0 +1,78 @@ +package models + +import ( + "context" + "math" + + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/cpumem/schedule" + "github.com/projecteru2/core/resources/cpumem/types" +) + +// GetNodesDeployCapacity . +func (c *CPUMem) GetNodesDeployCapacity(ctx context.Context, nodes []string, opts *types.WorkloadResourceOpts) (map[string]*types.NodeCapacityInfo, int, error) { + if err := opts.Validate(); err != nil { + logrus.Errorf("[GetNodesDeployCapacity] invalid resource opts %+v, err: %v", opts, err) + return nil, 0, err + } + + capacityInfoMap := map[string]*types.NodeCapacityInfo{} + total := 0 + for _, node := range nodes { + resourceInfo, err := c.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[GetNodesDeployCapacity] failed to get resource info of node %v, err: %v", node, err) + return nil, 0, err + } + capacityInfo := c.doGetNodeCapacityInfo(node, resourceInfo, opts) + if capacityInfo.Capacity > 0 { + capacityInfoMap[node] = capacityInfo + if total == math.MaxInt || capacityInfo.Capacity == math.MaxInt { + total = math.MaxInt + } else { + total += capacityInfo.Capacity + } + } + } + + return capacityInfoMap, total, nil +} + +func (c *CPUMem) doGetNodeCapacityInfo(node string, resourceInfo *types.NodeResourceInfo, opts *types.WorkloadResourceOpts) *types.NodeCapacityInfo { + availableResourceArgs := resourceInfo.GetAvailableResource() + + capacityInfo := &types.NodeCapacityInfo{ + Node: node, + Weight: 1, + } + + // if cpu-bind is not required, then returns capacity by memory + if !opts.CPUBind { + // check if cpu is enough + if opts.CPURequest > float64(len(resourceInfo.Capacity.CPUMap)) { + return capacityInfo + } + + // calculate by memory request + if opts.MemRequest == 0 { + capacityInfo.Capacity = math.MaxInt + capacityInfo.Rate = 0 + } else { + capacityInfo.Capacity = int(availableResourceArgs.Memory / opts.MemRequest) + capacityInfo.Rate = float64(opts.MemRequest) / float64(resourceInfo.Capacity.Memory) + } + capacityInfo.Usage = float64(resourceInfo.Usage.Memory) / float64(resourceInfo.Capacity.Memory) + + return capacityInfo + } + + // if cpu-bind is required, then returns capacity by cpu scheduling + cpuPlans := schedule.GetCPUPlans(resourceInfo, nil, c.Config.Scheduler.ShareBase, c.Config.Scheduler.MaxShare, opts) + capacityInfo.Capacity = len(cpuPlans) + capacityInfo.Usage = resourceInfo.Usage.CPU / resourceInfo.Capacity.CPU + capacityInfo.Rate = opts.CPURequest / resourceInfo.Capacity.CPU + capacityInfo.Weight = 100 + + return capacityInfo +} diff --git a/resources/cpumem/models/capacity_test.go b/resources/cpumem/models/capacity_test.go new file mode 100644 index 000000000..70c2dbab7 --- /dev/null +++ b/resources/cpumem/models/capacity_test.go @@ -0,0 +1,302 @@ +package models + +import ( + "context" + "errors" + "fmt" + "math" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/cpumem/types" + coretypes "github.com/projecteru2/core/types" +) + +func generateComplexNodes(t *testing.T, cpuMem *CPUMem) []string { + infos := []*types.NodeResourceInfo{ + { + Capacity: &types.NodeResourceArgs{ + CPU: 4, + CPUMap: types.CPUMap{ + "0": 100, + "1": 100, + "2": 100, + "3": 100, + }, + Memory: 12 * units.GiB, + }, + }, + { + Capacity: &types.NodeResourceArgs{ + CPU: 14, + CPUMap: types.CPUMap{ + "0": 100, + "1": 100, + "10": 100, + "11": 100, + "12": 100, + "13": 100, + "2": 100, + "3": 100, + "4": 100, + "5": 100, + "6": 100, + "7": 100, + "8": 100, + "9": 100, + }, + Memory: 12 * units.GiB, + }, + }, + { + Capacity: &types.NodeResourceArgs{ + CPU: 12, + CPUMap: types.CPUMap{ + "0": 100, + "1": 100, + "10": 100, + "11": 100, + "2": 100, + "3": 100, + "4": 100, + "5": 100, + "6": 100, + "7": 100, + "8": 100, + "9": 100, + }, + Memory: 12 * units.GiB, + }, + }, + { + Capacity: &types.NodeResourceArgs{ + CPU: 18, + CPUMap: types.CPUMap{ + "0": 100, + "1": 100, + "10": 100, + "11": 100, + "12": 100, + "13": 100, + "14": 100, + "15": 100, + "16": 100, + "17": 100, + "2": 100, + "3": 100, + "4": 100, + "5": 100, + "6": 100, + "7": 100, + "8": 100, + "9": 100, + }, + Memory: 12 * units.GiB, + }, + }, + { + Capacity: &types.NodeResourceArgs{ + CPU: 8, + CPUMap: types.CPUMap{ + "0": 100, + "1": 100, + "2": 100, + "3": 100, + "4": 100, + "5": 100, + "6": 100, + "7": 100, + }, + Memory: 12 * units.GiB, + }, + }, + } + nodes := []string{} + for i, info := range infos { + nodeName := fmt.Sprintf("node%d", i) + assert.Nil(t, cpuMem.doSetNodeResourceInfo(context.Background(), nodeName, info)) + nodes = append(nodes, nodeName) + } + return nodes +} + +func TestGetNodesCapacityWithCPUBinding(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 2, 2, 4*units.GiB, 100) + + // non-existent node + _, total, err := cpuMem.GetNodesDeployCapacity(ctx, []string{"xxx"}, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 0.5, + MemRequest: 1, + }) + assert.True(t, errors.Is(err, coretypes.ErrBadCount)) + + _, total, err = cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 0.5, + MemRequest: 1, + }) + assert.Nil(t, err) + assert.True(t, total >= 1) + + _, total, err = cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 2, + MemRequest: 1, + }) + assert.Nil(t, err) + assert.True(t, total < 3) + + _, total, err = cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 3, + MemRequest: 1, + }) + assert.Nil(t, err) + assert.True(t, total < 2) + + _, total, err = cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1, + MemRequest: 1, + }) + assert.Nil(t, err) + assert.True(t, total < 5) +} + +func TestComplexNodes(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateComplexNodes(t, cpuMem) + _, total, err := cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.7, + MemRequest: 1, + }) + assert.Nil(t, err) + assert.Equal(t, 28, total) +} + +func TestCPUNodesWithMemoryLimit(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 2, 2, 1024, 100) + _, total, err := cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 0.1, + MemRequest: 1024, + }) + assert.Nil(t, err) + assert.Equal(t, total, 2) + + _, total, err = cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 0.1, + MemRequest: 1025, + }) + assert.Nil(t, err) + assert.Equal(t, total, 0) +} + +func TestCPUNodesWithMaxShareLimit(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + cpuMem.Config.Scheduler.MaxShare = 2 + + nodes := generateNodes(t, cpuMem, 1, 6, 12*units.GiB, 100) + _, total, err := cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.7, + MemRequest: 1, + }) + assert.Nil(t, err) + assert.Equal(t, total, 2) + + nodeResourceInfo := &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: 4, + CPUMap: types.CPUMap{"0": 0, "1": 0, "2": 100, "3": 100}, + Memory: 12 * units.GiB, + }} + assert.Nil(t, nodeResourceInfo.Validate()) + _, _, err = cpuMem.doAllocByCPU(nodeResourceInfo, 1, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.2, + MemRequest: 1, + }) + assert.Nil(t, err) +} + +func BenchmarkGetNodesCapacity(b *testing.B) { + b.StopTimer() + t := &testing.T{} + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 10000, 24, 128*units.GiB, 100) + b.StartTimer() + for i := 0; i < b.N; i++ { + _, _, err := cpuMem.GetNodesDeployCapacity(context.Background(), nodes, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.3, + MemRequest: 1, + }) + assert.Nil(b, err) + } +} + +func TestGetNodesCapacityByMemory(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 2, 2, 4*units.GiB, 100) + + // negative memory + _, _, err := cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: false, + CPURequest: 0, + MemRequest: -1, + }) + assert.True(t, errors.Is(err, types.ErrInvalidMemory)) + + // cpu + mem + _, total, err := cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: false, + CPURequest: 1, + MemRequest: 512 * units.MiB, + }) + assert.Nil(t, err) + assert.Equal(t, total, 16) + + // unlimited cpu + _, total, err = cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: false, + CPURequest: 0, + MemRequest: 512 * units.MiB, + }) + assert.Nil(t, err) + assert.Equal(t, total, 16) + + // insufficient cpu + _, total, err = cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: false, + CPURequest: 3, + MemRequest: 512 * units.MiB, + }) + assert.Nil(t, err) + assert.Equal(t, total, 0) + + // mem_request == 0 + _, total, err = cpuMem.GetNodesDeployCapacity(ctx, nodes, &types.WorkloadResourceOpts{ + CPUBind: false, + CPURequest: 1, + MemRequest: 0, + }) + assert.Nil(t, err) + assert.Equal(t, total, math.MaxInt) +} diff --git a/resources/cpumem/models/cpumem.go b/resources/cpumem/models/cpumem.go new file mode 100644 index 000000000..8d7498ea1 --- /dev/null +++ b/resources/cpumem/models/cpumem.go @@ -0,0 +1,27 @@ +package models + +import ( + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/store/etcdv3/meta" + coretypes "github.com/projecteru2/core/types" +) + +// CPUMem manages cpu and memory +type CPUMem struct { + Config coretypes.Config + store meta.KV +} + +func NewCPUMem(config coretypes.Config) (*CPUMem, error) { + c := &CPUMem{Config: config} + var err error + if len(config.Etcd.Machines) > 0 { + c.store, err = meta.NewETCD(config.Etcd, nil) + if err != nil { + logrus.Errorf("[NewCPUMem] failed to create etcd client, err: %v", err) + return nil, err + } + } + return c, nil +} diff --git a/resources/cpumem/models/cpumem_test.go b/resources/cpumem/models/cpumem_test.go new file mode 100644 index 000000000..13b46ca55 --- /dev/null +++ b/resources/cpumem/models/cpumem_test.go @@ -0,0 +1,88 @@ +package models + +import ( + "context" + "fmt" + "strconv" + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/cpumem/types" + "github.com/projecteru2/core/store/etcdv3/meta" + coretypes "github.com/projecteru2/core/types" +) + +func newTestCPUMem(t *testing.T) *CPUMem { + config := coretypes.Config{ + Etcd: coretypes.EtcdConfig{ + Prefix: "/cpumem", + }, + Scheduler: coretypes.SchedConfig{ + MaxShare: -1, + ShareBase: 100, + }, + } + cpuMem, err := NewCPUMem(config) + assert.Nil(t, err) + store, err := meta.NewETCD(config.Etcd, t) + assert.Nil(t, err) + cpuMem.store = store + return cpuMem +} + +func generateNodeResourceInfos(t *testing.T, nums int, cores int, memory int64, shares int) []*types.NodeResourceInfo { + infos := []*types.NodeResourceInfo{} + for i := 0; i < nums; i++ { + cpuMap := types.CPUMap{} + for c := 0; c < cores; c++ { + cpuMap[strconv.Itoa(c)] = shares + } + + info := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + CPU: float64(cores), + CPUMap: cpuMap, + Memory: memory, + }, + Usage: nil, + } + assert.Nil(t, info.Validate()) + + infos = append(infos, info) + } + return infos +} + +func generateNodes(t *testing.T, cpuMem *CPUMem, nums int, cores int, memory int64, shares int) []string { + nodes := []string{} + infos := generateNodeResourceInfos(t, nums, cores, memory, shares) + + for i, info := range infos { + nodeName := fmt.Sprintf("node%d", i) + assert.Nil(t, cpuMem.doSetNodeResourceInfo(context.Background(), nodeName, info)) + nodes = append(nodes, nodeName) + } + + return nodes +} + +func TestNewCPUMem(t *testing.T) { + config := coretypes.Config{ + Etcd: coretypes.EtcdConfig{ + Machines: []string{"invalid-address"}, + }, + Scheduler: coretypes.SchedConfig{ + MaxShare: -1, + ShareBase: 100, + }, + } + cpuMem, err := NewCPUMem(config) + assert.Nil(t, err) + + ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + _, err = cpuMem.store.Put(ctx, "/test", "test") + assert.NotNil(t, err) +} diff --git a/resources/cpumem/models/idle.go b/resources/cpumem/models/idle.go new file mode 100644 index 000000000..111678ce3 --- /dev/null +++ b/resources/cpumem/models/idle.go @@ -0,0 +1,33 @@ +package models + +import ( + "context" + "math" + + "github.com/sirupsen/logrus" +) + +const priority = 100 + +// GetMostIdleNode . +func (c *CPUMem) GetMostIdleNode(ctx context.Context, nodes []string) (string, int, error) { + var mostIdleNode string + var minIdle = math.MaxFloat64 + + for _, node := range nodes { + resourceInfo, err := c.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[GetMostIdleNode] failed to get node resource info") + return "", 0, err + } + idle := float64(resourceInfo.Usage.CPUMap.TotalPieces()) / float64(resourceInfo.Capacity.CPUMap.TotalPieces()) + idle += float64(resourceInfo.Usage.Memory) / float64(resourceInfo.Capacity.Memory) + + if idle < minIdle { + mostIdleNode = node + minIdle = idle + } + } + + return mostIdleNode, priority, nil +} diff --git a/resources/cpumem/models/idle_test.go b/resources/cpumem/models/idle_test.go new file mode 100644 index 000000000..0b7ee46b3 --- /dev/null +++ b/resources/cpumem/models/idle_test.go @@ -0,0 +1,45 @@ +package models + +import ( + "context" + "fmt" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/cpumem/types" +) + +func TestGetMostIdleNode(t *testing.T) { + ctx := context.Background() + cpuMem := newTestCPUMem(t) + + infos := generateNodeResourceInfos(t, 2, 2, 2*units.GiB, 100) + infos[0].Usage = &types.NodeResourceArgs{ + CPU: 0, + CPUMap: types.CPUMap{}, + Memory: 0, + } + infos[1].Usage = &types.NodeResourceArgs{ + CPU: 1, + CPUMap: types.CPUMap{}, + Memory: 100, + } + + nodes := []string{} + + for i, info := range infos { + nodeName := fmt.Sprintf("node%d", i) + assert.Nil(t, cpuMem.doSetNodeResourceInfo(context.Background(), nodeName, info)) + nodes = append(nodes, nodeName) + } + + node, _, err := cpuMem.GetMostIdleNode(ctx, nodes) + assert.Nil(t, err) + assert.Equal(t, node, "node0") + + nodes = append(nodes, "node-x") + node, _, err = cpuMem.GetMostIdleNode(ctx, nodes) + assert.NotNil(t, err) +} diff --git a/resources/cpumem/models/info.go b/resources/cpumem/models/info.go new file mode 100644 index 000000000..07798dd12 --- /dev/null +++ b/resources/cpumem/models/info.go @@ -0,0 +1,215 @@ +package models + +import ( + "context" + "encoding/json" + "fmt" + + "github.com/sanity-io/litter" + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/cpumem/types" + "github.com/projecteru2/core/utils" +) + +const NodeResourceInfoKey = "/resource/cpumem/%s" + +// GetNodeResourceInfo . +func (c *CPUMem) GetNodeResourceInfo(ctx context.Context, node string, workloadResourceMap *types.WorkloadResourceArgsMap, fix bool) (*types.NodeResourceInfo, []string, error) { + resourceInfo, err := c.doGetNodeResourceInfo(ctx, node) + if err != nil { + return nil, nil, err + } + + diffs := []string{} + + totalResourceArgs := &types.WorkloadResourceArgs{ + CPUMap: types.CPUMap{}, + NUMAMemory: types.NUMAMemory{}, + } + + if workloadResourceMap != nil { + for _, args := range *workloadResourceMap { + totalResourceArgs.Add(args) + } + } + + totalResourceArgs.CPURequest = utils.Round(totalResourceArgs.CPURequest) + totalCPUUsage := utils.Round(resourceInfo.Usage.CPU) + if totalResourceArgs.CPURequest != totalCPUUsage { + diffs = append(diffs, fmt.Sprintf("node.CPUUsed != sum(workload.CPURequest): %.2f != %.2f", totalCPUUsage, totalResourceArgs.CPURequest)) + } + + for cpu := range resourceInfo.Capacity.CPUMap { + if totalResourceArgs.CPUMap[cpu] != resourceInfo.Usage.CPUMap[cpu] { + diffs = append(diffs, fmt.Sprintf("node.CPUMap[%v] != sum(workload.CPUMap[%v]): %v != %v", cpu, cpu, resourceInfo.Usage.CPUMap[cpu], totalResourceArgs.CPUMap[cpu])) + } + } + + for numaNodeID := range resourceInfo.Capacity.NUMAMemory { + if totalResourceArgs.NUMAMemory[numaNodeID] != resourceInfo.Usage.NUMAMemory[numaNodeID] { + diffs = append(diffs, fmt.Sprintf("node.NUMAMemory[%v] != sum(workload.NUMAMemory[%v]: %v != %v)", numaNodeID, numaNodeID, resourceInfo.Usage.NUMAMemory[numaNodeID], totalResourceArgs.NUMAMemory[numaNodeID])) + } + } + + if resourceInfo.Usage.Memory != totalResourceArgs.MemoryRequest { + diffs = append(diffs, fmt.Sprintf("node.MemoryUsed != sum(workload.MemoryRequest): %d != %d", resourceInfo.Usage.Memory, totalResourceArgs.MemoryRequest)) + } + + if fix { + resourceInfo.Usage = &types.NodeResourceArgs{ + CPU: totalResourceArgs.CPURequest, + CPUMap: totalResourceArgs.CPUMap, + Memory: totalResourceArgs.MemoryRequest, + NUMAMemory: totalResourceArgs.NUMAMemory, + } + if err = c.doSetNodeResourceInfo(ctx, node, resourceInfo); err != nil { + logrus.Warnf("[GetNodeResourceInfo] failed to fix node resource, err: %v", err) + diffs = append(diffs, "fix failed") + } + } + + return resourceInfo, diffs, nil +} + +// calculateNodeResourceArgs priority: node resource opts > node resource args > workload resource args list +func (c *CPUMem) calculateNodeResourceArgs(origin *types.NodeResourceArgs, nodeResourceOpts *types.NodeResourceOpts, nodeResourceArgs *types.NodeResourceArgs, workloadResourceArgs []*types.WorkloadResourceArgs, delta bool, incr bool) (res *types.NodeResourceArgs) { + if origin == nil || !delta { + res = (&types.NodeResourceArgs{}).DeepCopy() + } else { + res = origin.DeepCopy() + } + + if nodeResourceOpts != nil { + nodeResourceArgs := &types.NodeResourceArgs{ + CPU: float64(len(nodeResourceOpts.CPUMap)), + CPUMap: nodeResourceOpts.CPUMap, + Memory: nodeResourceOpts.Memory, + NUMAMemory: nodeResourceOpts.NUMAMemory, + NUMA: nodeResourceOpts.NUMA, + } + + if incr { + res.Add(nodeResourceArgs) + } else { + res.Sub(nodeResourceArgs) + } + return res + } + + if nodeResourceArgs != nil { + if incr { + res.Add(nodeResourceArgs) + } else { + res.Sub(nodeResourceArgs) + } + return res + } + + for _, args := range workloadResourceArgs { + nodeResourceArgs := &types.NodeResourceArgs{ + CPU: args.CPURequest, + CPUMap: args.CPUMap, + NUMAMemory: args.NUMAMemory, + Memory: args.MemoryRequest, + } + if incr { + res.Add(nodeResourceArgs) + } else { + res.Sub(nodeResourceArgs) + } + } + return res +} + +// SetNodeResourceUsage . +func (c *CPUMem) SetNodeResourceUsage(ctx context.Context, node string, nodeResourceOpts *types.NodeResourceOpts, nodeResourceArgs *types.NodeResourceArgs, workloadResourceArgs []*types.WorkloadResourceArgs, delta bool, incr bool) (before *types.NodeResourceArgs, after *types.NodeResourceArgs, err error) { + resourceInfo, err := c.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[SetNodeResourceInfo] failed to get resource info of node %v, err: %v", node, err) + return nil, nil, err + } + + before = resourceInfo.Usage.DeepCopy() + resourceInfo.Usage = c.calculateNodeResourceArgs(resourceInfo.Usage, nodeResourceOpts, nodeResourceArgs, workloadResourceArgs, delta, incr) + + if err := c.doSetNodeResourceInfo(ctx, node, resourceInfo); err != nil { + return nil, nil, err + } + return before, resourceInfo.Usage, nil +} + +// SetNodeResourceCapacity . +func (c *CPUMem) SetNodeResourceCapacity(ctx context.Context, node string, nodeResourceOpts *types.NodeResourceOpts, nodeResourceArgs *types.NodeResourceArgs, delta bool, incr bool) (before *types.NodeResourceArgs, after *types.NodeResourceArgs, err error) { + resourceInfo, err := c.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[SetNodeResourceInfo] failed to get resource info of node %v, err: %v", node, err) + return nil, nil, err + } + + before = resourceInfo.Capacity.DeepCopy() + if !delta && nodeResourceOpts != nil { + nodeResourceOpts.SkipEmpty(resourceInfo.Capacity) + } + + resourceInfo.Capacity = c.calculateNodeResourceArgs(resourceInfo.Capacity, nodeResourceOpts, nodeResourceArgs, nil, delta, incr) + + // add new cpu + for cpu := range resourceInfo.Capacity.CPUMap { + _, ok := resourceInfo.Usage.CPUMap[cpu] + if !ok { + resourceInfo.Usage.CPUMap[cpu] = 0 + } + } + + // delete cpus with no pieces + resourceInfo.RemoveEmptyCores() + + if err := c.doSetNodeResourceInfo(ctx, node, resourceInfo); err != nil { + return nil, nil, err + } + return before, resourceInfo.Capacity, nil +} + +// SetNodeResourceInfo . +func (c *CPUMem) SetNodeResourceInfo(ctx context.Context, node string, resourceCapacity *types.NodeResourceArgs, resourceUsage *types.NodeResourceArgs) error { + resourceInfo := &types.NodeResourceInfo{ + Capacity: resourceCapacity, + Usage: resourceUsage, + } + + return c.doSetNodeResourceInfo(ctx, node, resourceInfo) +} + +func (c *CPUMem) doGetNodeResourceInfo(ctx context.Context, node string) (*types.NodeResourceInfo, error) { + resourceInfo := &types.NodeResourceInfo{} + resp, err := c.store.GetOne(ctx, fmt.Sprintf(NodeResourceInfoKey, node)) + if err != nil { + logrus.Errorf("[doGetNodeResourceInfo] failed to get node resource info of node %v, err: %v", node, err) + return nil, err + } + if err = json.Unmarshal(resp.Value, resourceInfo); err != nil { + logrus.Errorf("[doGetNodeResourceInfo] failed to unmarshal node resource info of node %v, err: %v", node, err) + return nil, err + } + return resourceInfo, nil +} + +func (c *CPUMem) doSetNodeResourceInfo(ctx context.Context, node string, resourceInfo *types.NodeResourceInfo) error { + if err := resourceInfo.Validate(); err != nil { + logrus.Errorf("[doSetNodeResourceInfo] invalid resource info %v, err: %v", litter.Sdump(resourceInfo), err) + return err + } + + data, err := json.Marshal(resourceInfo) + if err != nil { + logrus.Errorf("[doSetNodeResourceInfo] faield to marshal resource info %+v, err: %v", resourceInfo, err) + return err + } + + if _, err = c.store.Put(ctx, fmt.Sprintf(NodeResourceInfoKey, node), string(data)); err != nil { + logrus.Errorf("[doSetNodeResourceInfo] faield to put resource info %+v, err: %v", resourceInfo, err) + return err + } + return nil +} diff --git a/resources/cpumem/models/info_test.go b/resources/cpumem/models/info_test.go new file mode 100644 index 000000000..86103979d --- /dev/null +++ b/resources/cpumem/models/info_test.go @@ -0,0 +1,311 @@ +package models + +import ( + "context" + "errors" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/cpumem/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestGetNodeResourceInfo(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 1, 2, 4*units.GiB, 100) + node := nodes[0] + + // invalid node + _, _, err := cpuMem.GetNodeResourceInfo(ctx, "xxx", nil, false) + assert.True(t, errors.Is(err, coretypes.ErrBadCount)) + + resourceInfo, diffs, err := cpuMem.GetNodeResourceInfo(ctx, node, nil, false) + assert.Nil(t, err) + assert.Equal(t, 0, len(diffs)) + + resourceInfo.Capacity.NUMA = types.NUMA{"0": "0", "1": "1"} + resourceInfo.Capacity.NUMAMemory = types.NUMAMemory{"0": 2 * units.GiB, "1": 2 * units.GiB} + + assert.Nil(t, cpuMem.SetNodeResourceInfo(ctx, node, resourceInfo.Capacity, resourceInfo.Usage)) + + resourceInfo, diffs, err = cpuMem.GetNodeResourceInfo(ctx, node, &types.WorkloadResourceArgsMap{ + "x-workload": { + CPURequest: 2, + CPUMap: types.CPUMap{"0": 100, "1": 100}, + MemoryRequest: 2 * units.GiB, + NUMAMemory: types.NUMAMemory{"0": units.GiB, "1": units.GiB}, + }, + }, true) + assert.Nil(t, err) + assert.Equal(t, 6, len(diffs)) + assert.Equal(t, resourceInfo.Usage, &types.NodeResourceArgs{ + CPU: 2, + CPUMap: types.CPUMap{"0": 100, "1": 100}, + Memory: 2 * units.GiB, + NUMAMemory: types.NUMAMemory{"0": units.GiB, "1": units.GiB}, + NUMA: types.NUMA{}, + }) +} + +func TestSetNodeResourceInfo(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 1, 2, 4*units.GiB, 100) + node := nodes[0] + + resourceInfo, _, err := cpuMem.GetNodeResourceInfo(ctx, node, nil, false) + assert.Nil(t, err) + + err = cpuMem.SetNodeResourceInfo(ctx, "node-x", resourceInfo.Capacity, resourceInfo.Usage) + assert.Nil(t, err) +} + +func TestSetNodeResourceUsage(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 1, 2, 4*units.GiB, 100) + node := nodes[0] + + _, _, err := cpuMem.GetNodeResourceInfo(ctx, node, nil, false) + assert.Nil(t, err) + + var after *types.NodeResourceArgs + + nodeResourceOpts := &types.NodeResourceOpts{ + CPUMap: map[string]int{ + "0": 100, + "1": 100, + }, + Memory: 2 * units.GiB, + } + + nodeResourceArgs := &types.NodeResourceArgs{ + CPU: 2, + CPUMap: map[string]int{ + "0": 100, + "1": 100, + }, + Memory: 2 * units.GiB, + } + + workloadResourceArgs := []*types.WorkloadResourceArgs{{ + CPURequest: 2, + CPUMap: map[string]int{ + "0": 100, + "1": 100, + }, + MemoryRequest: 2 * units.GiB, + }, + } + + originResourceUsage := &types.NodeResourceArgs{ + CPU: 0, + CPUMap: map[string]int{ + "0": 0, + "1": 0, + }, + Memory: 0, + NUMAMemory: types.NUMAMemory{}, + NUMA: types.NUMA{}, + } + + afterSetNodeResourceUsageDelta := &types.NodeResourceArgs{ + CPU: 2, + CPUMap: map[string]int{ + "0": 100, + "1": 100, + }, + Memory: 2 * units.GiB, + NUMAMemory: types.NUMAMemory{}, + NUMA: types.NUMA{}, + } + + _, after, err = cpuMem.SetNodeResourceUsage(ctx, node, nodeResourceOpts, nil, nil, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDelta) + + _, after, err = cpuMem.SetNodeResourceUsage(ctx, node, nodeResourceOpts, nil, nil, true, false) + assert.Nil(t, err) + assert.Equal(t, originResourceUsage, after) + + _, after, err = cpuMem.SetNodeResourceUsage(ctx, node, nil, nodeResourceArgs, nil, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDelta) + + _, after, err = cpuMem.SetNodeResourceUsage(ctx, node, nil, nodeResourceArgs, nil, true, false) + assert.Nil(t, err) + assert.Equal(t, originResourceUsage, after) + + _, after, err = cpuMem.SetNodeResourceUsage(ctx, node, nil, nil, workloadResourceArgs, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDelta) + + _, after, err = cpuMem.SetNodeResourceUsage(ctx, node, nil, nil, workloadResourceArgs, true, false) + assert.Nil(t, err) + assert.Equal(t, originResourceUsage, after) + + _, after, err = cpuMem.SetNodeResourceUsage(ctx, node, nil, nil, nil, true, false) + assert.Nil(t, err) + assert.Equal(t, originResourceUsage, after) + + _, after, err = cpuMem.SetNodeResourceUsage(ctx, node, nil, nodeResourceArgs, nil, false, true) + assert.Nil(t, err) + assert.Equal(t, nodeResourceArgs.DeepCopy(), after.DeepCopy()) +} + +func TestSetNodeResourceCapacity(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 1, 2, 2*units.GiB, 100) + node := nodes[0] + + _, _, err := cpuMem.GetNodeResourceInfo(ctx, node, nil, false) + assert.Nil(t, err) + + var after *types.NodeResourceArgs + + nodeResourceOptsDelta := &types.NodeResourceOpts{ + CPUMap: map[string]int{ + "2": 100, + "3": 100, + }, + Memory: 0, + NUMAMemory: types.NUMAMemory{ + "0": units.GiB, + "1": units.GiB, + }, + NUMA: types.NUMA{ + "0": "0", + "1": "0", + "2": "1", + "3": "1", + }, + } + + nodeResourceArgsDelta := &types.NodeResourceArgs{ + CPU: 2, + CPUMap: map[string]int{ + "2": 100, + "3": 100, + }, + Memory: 0, + NUMAMemory: types.NUMAMemory{ + "0": units.GiB, + "1": units.GiB, + }, + NUMA: types.NUMA{ + "0": "0", + "1": "0", + "2": "1", + "3": "1", + }, + } + + originResourceCapacity := &types.NodeResourceArgs{ + CPU: 2, + CPUMap: map[string]int{ + "0": 100, + "1": 100, + }, + Memory: 2 * units.GiB, + NUMAMemory: types.NUMAMemory{}, + NUMA: types.NUMA{}, + } + + originResourceCapacityWithNUMAChanged := &types.NodeResourceArgs{ + CPU: 2, + CPUMap: map[string]int{ + "0": 100, + "1": 100, + }, + Memory: 2 * units.GiB, + NUMAMemory: types.NUMAMemory{ + "0": 0, + "1": 0, + }, + NUMA: types.NUMA{ + "0": "0", + "1": "0", + "2": "1", + "3": "1", + }, + } + + originResourceCapacityOpts := &types.NodeResourceOpts{ + CPUMap: map[string]int{ + "0": 100, + "1": 100, + }, + Memory: 2 * units.GiB, + NUMAMemory: types.NUMAMemory{}, + NUMA: types.NUMA{}, + RawParams: coretypes.RawParams{ + "cpu": "", + }, + } + + afterSetNodeResourceUsageDeltaWithNUMAChanged := &types.NodeResourceArgs{ + CPU: 4, + CPUMap: map[string]int{ + "0": 100, + "1": 100, + "2": 100, + "3": 100, + }, + Memory: 2 * units.GiB, + NUMAMemory: types.NUMAMemory{ + "0": units.GiB, + "1": units.GiB, + }, + NUMA: types.NUMA{ + "0": "0", + "1": "0", + "2": "1", + "3": "1", + }, + } + + afterSetNodeResourceUsageDeltaWithNUMANotChanged := &types.NodeResourceArgs{ + CPU: 4, + CPUMap: map[string]int{ + "0": 100, + "1": 100, + "2": 100, + "3": 100, + }, + Memory: 2 * units.GiB, + NUMAMemory: types.NUMAMemory{}, + NUMA: types.NUMA{}, + } + + _, after, err = cpuMem.SetNodeResourceCapacity(ctx, node, nodeResourceOptsDelta, nil, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDeltaWithNUMAChanged) + + _, after, err = cpuMem.SetNodeResourceCapacity(ctx, node, nodeResourceOptsDelta, nil, true, false) + assert.Nil(t, err) + assert.Equal(t, after, originResourceCapacityWithNUMAChanged) + + _, after, err = cpuMem.SetNodeResourceCapacity(ctx, node, nil, nodeResourceArgsDelta, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDeltaWithNUMAChanged) + + _, after, err = cpuMem.SetNodeResourceCapacity(ctx, node, nil, nodeResourceArgsDelta, true, false) + assert.Nil(t, err) + assert.Equal(t, after, originResourceCapacityWithNUMAChanged) + + _, after, err = cpuMem.SetNodeResourceCapacity(ctx, node, nil, afterSetNodeResourceUsageDeltaWithNUMANotChanged, false, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDeltaWithNUMANotChanged) + + _, after, err = cpuMem.SetNodeResourceCapacity(ctx, node, originResourceCapacityOpts, nil, false, true) + assert.Nil(t, err) + assert.Equal(t, after, originResourceCapacity) +} diff --git a/resources/cpumem/models/metrics.go b/resources/cpumem/models/metrics.go new file mode 100644 index 000000000..26075da20 --- /dev/null +++ b/resources/cpumem/models/metrics.go @@ -0,0 +1,72 @@ +package models + +import ( + "fmt" + "strings" + + "github.com/projecteru2/core/resources/cpumem/types" +) + +// GetMetricsDescription . +func (c *CPUMem) GetMetricsDescription() []map[string]interface{} { + return []map[string]interface{}{ + { + "name": "memory_capacity", + "help": "node available memory.", + "type": "gauge", + "labels": []string{"podname", "nodename"}, + }, + { + "name": "memory_used", + "help": "node used memory.", + "type": "gauge", + "labels": []string{"podname", "nodename"}, + }, + { + "name": "cpu_map", + "help": "node available cpu.", + "type": "gauge", + "labels": []string{"podname", "nodename", "cpuid"}, + }, + { + "name": "cpu_used", + "help": "node used cpu.", + "type": "gauge", + "labels": []string{"podname", "nodename"}, + }, + } +} + +func (c *CPUMem) ResolveNodeResourceInfoToMetrics(podName string, nodeName string, nodeResourceCapacity *types.NodeResourceArgs, nodeResourceUsage *types.NodeResourceArgs) []map[string]interface{} { + cleanedNodeName := strings.ReplaceAll(nodeName, ".", "_") + metrics := []map[string]interface{}{ + { + "name": "memory_capacity", + "labels": []string{podName, nodeName}, + "value": fmt.Sprintf("%v", nodeResourceCapacity.Memory), + "key": fmt.Sprintf("core.node.%s.memory", cleanedNodeName), + }, + { + "name": "memory_used", + "labels": []string{podName, nodeName}, + "value": fmt.Sprintf("%v", nodeResourceUsage.Memory), + "key": fmt.Sprintf("core.node.%s.memory.used", cleanedNodeName), + }, + { + "name": "cpu_used", + "labels": []string{podName, nodeName}, + "value": fmt.Sprintf("%v", nodeResourceUsage.CPU), + "key": fmt.Sprintf("core.node.%s.cpu.used", cleanedNodeName), + }, + } + + for cpuID, pieces := range nodeResourceUsage.CPUMap { + metrics = append(metrics, map[string]interface{}{ + "name": "cpu_map", + "labels": []string{podName, nodeName, cpuID}, + "value": fmt.Sprintf("%v", pieces), + "key": fmt.Sprintf("core.node.%s.cpu.%s", cleanedNodeName, cpuID), + }) + } + return metrics +} diff --git a/resources/cpumem/models/node.go b/resources/cpumem/models/node.go new file mode 100644 index 000000000..ea6d6572e --- /dev/null +++ b/resources/cpumem/models/node.go @@ -0,0 +1,69 @@ +package models + +import ( + "context" + "errors" + "fmt" + + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/cpumem/types" + coretypes "github.com/projecteru2/core/types" +) + +// AddNode . +func (c *CPUMem) AddNode(ctx context.Context, node string, resourceOpts *types.NodeResourceOpts) (*types.NodeResourceInfo, error) { + if _, err := c.doGetNodeResourceInfo(ctx, node); err != nil { + if !errors.Is(err, coretypes.ErrBadCount) { + logrus.Errorf("[AddNode] failed to get resource info of node %v, err: %v", node, err) + return nil, err + } + } else { + return nil, types.ErrNodeExists + } + + resourceInfo := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + CPU: float64(len(resourceOpts.CPUMap)), + CPUMap: resourceOpts.CPUMap, + Memory: resourceOpts.Memory, + NUMAMemory: resourceOpts.NUMAMemory, + NUMA: resourceOpts.NUMA, + }, + Usage: &types.NodeResourceArgs{ + CPU: 0, + CPUMap: types.CPUMap{}, + Memory: 0, + NUMAMemory: types.NUMAMemory{}, + }, + } + + // if NUMA is set but NUMAMemory is not set + // then divide memory equally according to the number of numa nodes + if len(resourceOpts.NUMA) > 0 && resourceOpts.NUMAMemory == nil { + averageMemory := resourceOpts.Memory / int64(len(resourceOpts.NUMA)) + resourceInfo.Capacity.NUMAMemory = types.NUMAMemory{} + for _, numaNodeID := range resourceOpts.NUMA { + resourceInfo.Capacity.NUMAMemory[numaNodeID] = averageMemory + } + } + + for cpu := range resourceOpts.CPUMap { + resourceInfo.Usage.CPUMap[cpu] = 0 + } + + for numaNodeID := range resourceOpts.NUMA { + resourceInfo.Usage.NUMAMemory[numaNodeID] = 0 + } + + return resourceInfo, c.doSetNodeResourceInfo(ctx, node, resourceInfo) +} + +// RemoveNode . +func (c *CPUMem) RemoveNode(ctx context.Context, node string) error { + if _, err := c.store.Delete(ctx, fmt.Sprintf(NodeResourceInfoKey, node)); err != nil { + logrus.Errorf("[doSetNodeResourceInfo] faield to delete node %v, err: %v", node, err) + return err + } + return nil +} diff --git a/resources/cpumem/models/node_test.go b/resources/cpumem/models/node_test.go new file mode 100644 index 000000000..64507874b --- /dev/null +++ b/resources/cpumem/models/node_test.go @@ -0,0 +1,64 @@ +package models + +import ( + "context" + "errors" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/cpumem/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestAddNode(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 1, 1, 1, 100) + + node := "node" + resourceOpts := &types.NodeResourceOpts{ + CPUMap: types.CPUMap{"0": 100, "1": 100}, + Memory: 4 * units.GiB, + NUMA: types.NUMA{"0": "0", "1": "1"}, + } + + // existent node + _, err := cpuMem.AddNode(ctx, nodes[0], resourceOpts) + assert.Equal(t, err, types.ErrNodeExists) + + // normal case + resourceInfo, err := cpuMem.AddNode(ctx, node, resourceOpts) + assert.Nil(t, err) + assert.Equal(t, resourceInfo, &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + CPU: 2, + CPUMap: types.CPUMap{"0": 100, "1": 100}, + Memory: 4 * units.GiB, + NUMAMemory: types.NUMAMemory{"0": 2 * units.GiB, "1": 2 * units.GiB}, + NUMA: types.NUMA{"0": "0", "1": "1"}, + }, + Usage: &types.NodeResourceArgs{ + CPU: 0, + CPUMap: types.CPUMap{"0": 0, "1": 0}, + Memory: 0, + NUMAMemory: types.NUMAMemory{"0": 0, "1": 0}, + NUMA: types.NUMA{}, + }, + }) +} + +func TestRemoveNode(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + nodes := generateNodes(t, cpuMem, 1, 1, 1, 100) + + assert.Nil(t, cpuMem.RemoveNode(ctx, nodes[0])) + _, _, err := cpuMem.GetNodeResourceInfo(ctx, nodes[0], nil, false) + assert.True(t, errors.Is(err, coretypes.ErrBadCount)) + + assert.Nil(t, cpuMem.RemoveNode(ctx, "xxx")) +} diff --git a/resources/cpumem/models/realloc.go b/resources/cpumem/models/realloc.go new file mode 100644 index 000000000..54479df5d --- /dev/null +++ b/resources/cpumem/models/realloc.go @@ -0,0 +1,88 @@ +package models + +import ( + "context" + + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/cpumem/schedule" + "github.com/projecteru2/core/resources/cpumem/types" +) + +// GetReallocArgs . +func (c *CPUMem) GetReallocArgs(ctx context.Context, node string, originResourceArgs *types.WorkloadResourceArgs, resourceOpts *types.WorkloadResourceOpts) (*types.EngineArgs, *types.WorkloadResourceArgs, *types.WorkloadResourceArgs, error) { + if resourceOpts.KeepCPUBind { + resourceOpts.CPUBind = len(originResourceArgs.CPUMap) > 0 + } + + resourceInfo, err := c.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[GetReallocArgs] failed to get resource info of node %v, err: %v", node, err) + return nil, nil, nil, err + } + + // put resources back into the resource pool + resourceInfo.Usage.Sub(&types.NodeResourceArgs{ + CPU: originResourceArgs.CPURequest, + CPUMap: originResourceArgs.CPUMap, + Memory: originResourceArgs.MemoryRequest, + NUMAMemory: originResourceArgs.NUMAMemory, + }) + + finalResourceOpts := &types.WorkloadResourceOpts{ + CPUBind: resourceOpts.CPUBind, + CPURequest: resourceOpts.CPURequest + originResourceArgs.CPURequest, + CPULimit: resourceOpts.CPULimit + originResourceArgs.CPULimit, + MemRequest: resourceOpts.MemRequest + originResourceArgs.MemoryRequest, + MemLimit: resourceOpts.MemLimit + originResourceArgs.MemoryLimit, + } + + if err = finalResourceOpts.Validate(); err != nil { + return nil, nil, nil, err + } + + // if cpu was specified before, try to ensure cpu affinity + var cpuMap types.CPUMap + var numaNodeID string + var numaMemory types.NUMAMemory + + if resourceOpts.CPUBind { + cpuPlans := schedule.GetCPUPlans(resourceInfo, originResourceArgs.CPUMap, c.Config.Scheduler.ShareBase, c.Config.Scheduler.MaxShare, finalResourceOpts) + if len(cpuPlans) == 0 { + return nil, nil, nil, types.ErrInsufficientResource + } + + cpuPlan := cpuPlans[0] + cpuMap = cpuPlan.CPUMap + numaNodeID = cpuPlan.NUMANode + if len(numaNodeID) > 0 { + numaMemory = types.NUMAMemory{numaNodeID: finalResourceOpts.MemRequest} + } + } else { + if _, _, err = c.doAllocByMemory(resourceInfo, 1, finalResourceOpts); err != nil { + return nil, nil, nil, err + } + } + + engineArgs := &types.EngineArgs{ + CPU: finalResourceOpts.CPULimit, + CPUMap: cpuMap, + NUMANode: numaNodeID, + Memory: finalResourceOpts.MemLimit, + } + + finalWorkloadResourceArgs := &types.WorkloadResourceArgs{ + CPURequest: finalResourceOpts.CPURequest, + CPULimit: finalResourceOpts.CPULimit, + MemoryRequest: finalResourceOpts.MemRequest, + MemoryLimit: finalResourceOpts.MemLimit, + CPUMap: cpuMap, + NUMAMemory: numaMemory, + NUMANode: numaNodeID, + } + + deltaWorkloadResourceArgs := finalWorkloadResourceArgs.DeepCopy() + deltaWorkloadResourceArgs.Sub(originResourceArgs) + + return engineArgs, deltaWorkloadResourceArgs, finalWorkloadResourceArgs, nil +} diff --git a/resources/cpumem/models/realloc_test.go b/resources/cpumem/models/realloc_test.go new file mode 100644 index 000000000..088c7c705 --- /dev/null +++ b/resources/cpumem/models/realloc_test.go @@ -0,0 +1,102 @@ +package models + +import ( + "context" + "errors" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/cpumem/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestRealloc(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + node := "node" + resourceOpts := &types.NodeResourceOpts{ + CPUMap: types.CPUMap{"0": 100, "1": 100}, + Memory: 4 * units.GiB, + NUMA: types.NUMA{"0": "0", "1": "1"}, + } + + _, err := cpuMem.AddNode(ctx, node, resourceOpts) + assert.Nil(t, err) + + originResourceArgs := &types.WorkloadResourceArgs{ + CPURequest: 1, + CPULimit: 1, + MemoryRequest: units.GiB, + MemoryLimit: units.GiB, + CPUMap: types.CPUMap{"0": 100}, + NUMAMemory: types.NUMAMemory{"0": units.GiB}, + NUMANode: "0", + } + + // non-existent node + _, _, _, err = cpuMem.GetReallocArgs(ctx, "xxx", originResourceArgs, &types.WorkloadResourceOpts{}) + assert.True(t, errors.Is(err, coretypes.ErrBadCount)) + + // invalid resource opts + opts := &types.WorkloadResourceOpts{ + CPUBind: false, + KeepCPUBind: true, + CPURequest: -3, + CPULimit: 0, + MemRequest: 0, + MemLimit: 0, + } + _, _, _, err = cpuMem.GetReallocArgs(ctx, node, originResourceArgs, opts) + assert.True(t, errors.Is(err, types.ErrInvalidCPU)) + + // insufficient cpu + opts = &types.WorkloadResourceOpts{ + CPUBind: false, + KeepCPUBind: true, + CPURequest: 2, + CPULimit: 0, + MemRequest: 0, + MemLimit: 0, + } + _, _, _, err = cpuMem.GetReallocArgs(ctx, node, originResourceArgs, opts) + assert.True(t, errors.Is(err, types.ErrInsufficientResource)) + + // normal case (with cpu-bind) + opts = &types.WorkloadResourceOpts{ + CPUBind: false, + KeepCPUBind: true, + CPURequest: -0.5, + CPULimit: -0.5, + MemRequest: 0, + MemLimit: 0, + } + _, _, _, err = cpuMem.GetReallocArgs(ctx, node, originResourceArgs, opts) + assert.Nil(t, err) + + // normal case (without cpu-bind) + opts = &types.WorkloadResourceOpts{ + CPUBind: false, + KeepCPUBind: false, + CPURequest: 0, + CPULimit: 0, + MemRequest: 0, + MemLimit: 0, + } + _, _, _, err = cpuMem.GetReallocArgs(ctx, node, originResourceArgs, opts) + assert.Nil(t, err) + + // insufficient mem + opts = &types.WorkloadResourceOpts{ + CPUBind: false, + KeepCPUBind: false, + CPURequest: 0, + CPULimit: 0, + MemRequest: units.PiB, + MemLimit: units.PiB, + } + _, _, _, err = cpuMem.GetReallocArgs(ctx, node, originResourceArgs, opts) + assert.True(t, errors.Is(err, types.ErrInsufficientMem)) +} diff --git a/resources/cpumem/models/remap.go b/resources/cpumem/models/remap.go new file mode 100644 index 000000000..46d43e4f6 --- /dev/null +++ b/resources/cpumem/models/remap.go @@ -0,0 +1,52 @@ +package models + +import ( + "context" + + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/cpumem/types" +) + +// GetRemapArgs . +func (c *CPUMem) GetRemapArgs(ctx context.Context, node string, workloadResourceMap *types.WorkloadResourceArgsMap) (map[string]*types.EngineArgs, error) { + resourceInfo, err := c.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[GetRemapArgs] failed to get resource info of node %v, err: %v", node, err) + return nil, err + } + availableNodeResource := resourceInfo.GetAvailableResource() + + shareCPUMap := types.CPUMap{} + for cpu, pieces := range availableNodeResource.CPUMap { + if pieces >= c.Config.Scheduler.ShareBase { + shareCPUMap[cpu] = c.Config.Scheduler.ShareBase + } + } + + if len(shareCPUMap) == 0 { + for cpu := range resourceInfo.Capacity.CPUMap { + shareCPUMap[cpu] = c.Config.Scheduler.ShareBase + } + } + + engineArgsMap := map[string]*types.EngineArgs{} + + if workloadResourceMap == nil { + return engineArgsMap, nil + } + + for workloadID, workloadResourceArgs := range *workloadResourceMap { + // only process workloads without cpu binding + if len(workloadResourceArgs.CPUMap) == 0 { + engineArgsMap[workloadID] = &types.EngineArgs{ + CPU: workloadResourceArgs.CPULimit, + CPUMap: shareCPUMap, + NUMANode: "", + Memory: workloadResourceArgs.MemoryLimit, + Remap: true, + } + } + } + return engineArgsMap, nil +} diff --git a/resources/cpumem/models/remap_test.go b/resources/cpumem/models/remap_test.go new file mode 100644 index 000000000..9d822cec8 --- /dev/null +++ b/resources/cpumem/models/remap_test.go @@ -0,0 +1,107 @@ +package models + +import ( + "context" + "errors" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/cpumem/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestRemap(t *testing.T) { + ctx := context.Background() + + cpuMem := newTestCPUMem(t) + node := "node" + resourceInfo := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + CPU: 4, + CPUMap: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100}, + Memory: 4 * units.GiB, + }, + } + assert.Nil(t, cpuMem.doSetNodeResourceInfo(ctx, node, resourceInfo)) + + // non-existent node + _, err := cpuMem.GetRemapArgs(ctx, "xxx", nil) + assert.True(t, errors.Is(err, coretypes.ErrBadCount)) + + // normal case + workloadResourceMap := &types.WorkloadResourceArgsMap{ + "w1": { + CPUMap: nil, + CPURequest: 1, + CPULimit: 1, + MemoryRequest: units.GiB, + MemoryLimit: units.GiB, + }, + "w2": { + CPUMap: nil, + CPURequest: 1, + CPULimit: 1, + MemoryRequest: units.GiB, + MemoryLimit: units.GiB, + }, + "w3": { + CPUMap: types.CPUMap{"0": 100, "1": 100}, + CPURequest: 2, + CPULimit: 2, + MemoryRequest: units.GiB, + MemoryLimit: units.GiB, + }, + } + resourceInfo, _, err = cpuMem.GetNodeResourceInfo(ctx, node, workloadResourceMap, true) + assert.Nil(t, err) + + engineArgsMap, err := cpuMem.GetRemapArgs(ctx, node, workloadResourceMap) + assert.Nil(t, err) + assert.Equal(t, 2, len(engineArgsMap)) + assert.Equal(t, engineArgsMap["w1"], &types.EngineArgs{ + CPU: 1, + CPUMap: types.CPUMap{"2": 100, "3": 100}, + NUMANode: "", + Memory: units.GiB, + Remap: true, + }) + assert.Equal(t, engineArgsMap["w2"], &types.EngineArgs{ + CPU: 1, + CPUMap: types.CPUMap{"2": 100, "3": 100}, + NUMANode: "", + Memory: units.GiB, + Remap: true, + }) + + // empty share cpu map + (*workloadResourceMap)["w4"] = &types.WorkloadResourceArgs{ + CPUMap: types.CPUMap{"2": 100, "3": 100}, + CPURequest: 2, + CPULimit: 2, + MemoryRequest: units.GiB, + MemoryLimit: units.GiB, + } + + _, _, err = cpuMem.GetNodeResourceInfo(ctx, node, workloadResourceMap, true) + assert.Nil(t, err) + + engineArgsMap, err = cpuMem.GetRemapArgs(ctx, node, workloadResourceMap) + assert.Nil(t, err) + assert.Equal(t, 2, len(engineArgsMap)) + assert.Equal(t, engineArgsMap["w1"], &types.EngineArgs{ + CPU: 1, + CPUMap: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100}, + NUMANode: "", + Memory: units.GiB, + Remap: true, + }) + assert.Equal(t, engineArgsMap["w2"], &types.EngineArgs{ + CPU: 1, + CPUMap: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100}, + NUMANode: "", + Memory: units.GiB, + Remap: true, + }) +} diff --git a/resources/cpumem/schedule/schedule.go b/resources/cpumem/schedule/schedule.go new file mode 100644 index 000000000..3ee084820 --- /dev/null +++ b/resources/cpumem/schedule/schedule.go @@ -0,0 +1,346 @@ +package schedule + +import ( + "container/heap" + "sort" + "strconv" + + "github.com/projecteru2/core/resources/cpumem/types" +) + +type cpuCore struct { + id string + pieces int +} + +func (c *cpuCore) LessThan(c1 *cpuCore) bool { + if c.pieces == c1.pieces { + idI, _ := strconv.Atoi(c.id) + idJ, _ := strconv.Atoi(c1.id) + return idI < idJ + } + return c.pieces < c1.pieces +} + +type cpuCoreHeap []*cpuCore + +// Len . +func (c cpuCoreHeap) Len() int { + return len(c) +} + +// Less . +func (c cpuCoreHeap) Less(i, j int) bool { + return !c[i].LessThan(c[j]) +} + +// Swap . +func (c cpuCoreHeap) Swap(i, j int) { + c[i], c[j] = c[j], c[i] +} + +// Push . +func (c *cpuCoreHeap) Push(x interface{}) { + *c = append(*c, x.(*cpuCore)) +} + +// Pop . +func (c *cpuCoreHeap) Pop() interface{} { + old := *c + n := len(old) + x := old[n-1] + *c = old[:n-1] + return x +} + +type host struct { + shareBase int + maxFragmentCores int + fullCores []*cpuCore + fragmentCores []*cpuCore + affinity bool +} + +// CPUPlan . +type CPUPlan struct { + NUMANode string + CPUMap types.CPUMap +} + +func min(a, b int) int { + if a < b { + return a + } + return b +} + +// GetCPUPlans . +func GetCPUPlans(resourceInfo *types.NodeResourceInfo, originCPUMap types.CPUMap, shareBase int, maxFragmentCores int, resourceOpts *types.WorkloadResourceOpts) []*CPUPlan { + cpuPlans := []*CPUPlan{} + availableResourceArgs := resourceInfo.GetAvailableResource() + + numaCPUMap := map[string]types.CPUMap{} + for cpuID, numaNodeID := range resourceInfo.Capacity.NUMA { + if _, ok := numaCPUMap[numaNodeID]; !ok { + numaCPUMap[numaNodeID] = types.CPUMap{} + } + numaCPUMap[numaNodeID][cpuID] = availableResourceArgs.CPUMap[cpuID] + } + + // get cpu plan for each numa node + for numaNodeID, cpuMap := range numaCPUMap { + numaCPUPlans := doGetCPUPlans(originCPUMap, cpuMap, availableResourceArgs.NUMAMemory[numaNodeID], shareBase, maxFragmentCores, resourceOpts.CPURequest, resourceOpts.MemRequest) + for _, workloadCPUMap := range numaCPUPlans { + cpuPlans = append(cpuPlans, &CPUPlan{ + NUMANode: numaNodeID, + CPUMap: workloadCPUMap, + }) + availableResourceArgs.Sub(&types.NodeResourceArgs{ + CPU: resourceOpts.CPURequest, + CPUMap: workloadCPUMap, + Memory: resourceOpts.MemRequest, + NUMAMemory: types.NUMAMemory{numaNodeID: resourceOpts.MemRequest}, + }) + } + } + + // get cpu plan with the remaining resource + crossNUMACPUPlans := doGetCPUPlans(originCPUMap, availableResourceArgs.CPUMap, availableResourceArgs.Memory, shareBase, maxFragmentCores, resourceOpts.CPURequest, resourceOpts.MemRequest) + for _, workloadCPUMap := range crossNUMACPUPlans { + cpuPlans = append(cpuPlans, &CPUPlan{ + CPUMap: workloadCPUMap, + }) + } + + return cpuPlans +} + +// ensure that the old cpu core will still be allocated first +func reorderByAffinity(oldH, newH *host) { + oldFull := map[string]int{} + oldFragment := map[string]int{} + + for i, core := range oldH.fullCores { + oldFull[core.id] = i + 1 + } + for i, core := range oldH.fragmentCores { + oldFragment[core.id] = i + 1 + } + + sortFunc := func(orderMap map[string]int, cores []*cpuCore) func(i, j int) bool { + return func(i, j int) bool { + idxI := orderMap[cores[i].id] + idxJ := orderMap[cores[j].id] + + if idxI == 0 && idxJ == 0 { + return i < j + } + if idxI == 0 || idxJ == 0 { + return idxI > idxJ + } + return idxI < idxJ + } + } + + sort.SliceStable(newH.fullCores, sortFunc(oldFull, newH.fullCores)) + sort.SliceStable(newH.fragmentCores, sortFunc(oldFragment, newH.fragmentCores)) + newH.affinity = true +} + +// doGetCPUPlans . +func doGetCPUPlans(originCPUMap, availableCPUMap types.CPUMap, availableMemory int64, shareBase int, maxFragmentCores int, cpuRequest float64, memoryRequest int64) []types.CPUMap { + h := newHost(availableCPUMap, shareBase, maxFragmentCores) + + // affinity + if len(originCPUMap) > 0 { + originH := newHost(originCPUMap, shareBase, maxFragmentCores) + reorderByAffinity(originH, h) + } + + cpuPlans := h.getCPUPlans(cpuRequest) + if memoryRequest > 0 { + memoryCapacity := int(availableMemory / memoryRequest) + if memoryCapacity < len(cpuPlans) { + cpuPlans = cpuPlans[:memoryCapacity] + } + } + return cpuPlans +} + +func newHost(cpuMap types.CPUMap, shareBase int, maxFragmentCores int) *host { + h := &host{ + shareBase: shareBase, + maxFragmentCores: maxFragmentCores, + fullCores: []*cpuCore{}, + fragmentCores: []*cpuCore{}, + } + + for cpu, pieces := range cpuMap { + if pieces >= shareBase && pieces%shareBase == 0 { + h.fullCores = append(h.fullCores, &cpuCore{id: cpu, pieces: pieces}) + } else if pieces > 0 { + h.fragmentCores = append(h.fragmentCores, &cpuCore{id: cpu, pieces: pieces}) + } + } + + sortFunc := func(cores []*cpuCore) func(i, j int) bool { + return func(i, j int) bool { + // give priority to the CPU cores with higher load + return cores[i].LessThan(cores[j]) + } + } + + sort.SliceStable(h.fullCores, sortFunc(h.fullCores)) + sort.SliceStable(h.fragmentCores, sortFunc(h.fragmentCores)) + + return h +} + +func (h *host) getCPUPlans(cpuRequest float64) []types.CPUMap { + piecesRequest := int(cpuRequest * float64(h.shareBase)) + full := piecesRequest / h.shareBase + fragment := piecesRequest % h.shareBase + + maxFragmentCores := len(h.fullCores) + len(h.fragmentCores) - full + if h.maxFragmentCores == -1 || h.maxFragmentCores > maxFragmentCores { + h.maxFragmentCores = maxFragmentCores + } + + if fragment == 0 { + return h.getFullCPUPlans(h.fullCores, full) + } + + if full == 0 { + diff := h.maxFragmentCores - len(h.fragmentCores) + h.fragmentCores = append(h.fragmentCores, h.fullCores[:diff]...) + h.fullCores = h.fullCores[diff:] + return h.getFragmentCPUPlans(h.fragmentCores, fragment) + } + + fragmentCapacityMap := map[string]int{} + totalFragmentCapacity := 0 // for lazy loading + bestCPUPlans := [2][]types.CPUMap{h.getFullCPUPlans(h.fullCores, full), h.getFragmentCPUPlans(h.fragmentCores, fragment)} + bestCapacity := min(len(bestCPUPlans[0]), len(bestCPUPlans[1])) + + for _, core := range h.fullCores { + fragmentCapacityMap[core.id] = core.pieces / fragment + } + + for _, core := range h.fragmentCores { + fragmentCapacityMap[core.id] = core.pieces / fragment + totalFragmentCapacity += fragmentCapacityMap[core.id] + } + + for len(h.fragmentCores) < h.maxFragmentCores { + // convert a full core to fragment core + newFragmentCore := h.fullCores[0] + h.fragmentCores = append(h.fragmentCores, newFragmentCore) + h.fullCores = h.fullCores[1:] + totalFragmentCapacity += fragmentCapacityMap[newFragmentCore.id] + + fullCPUPlans := h.getFullCPUPlans(h.fullCores, full) + capacity := min(len(fullCPUPlans), totalFragmentCapacity) + if capacity > bestCapacity { + bestCPUPlans[0] = fullCPUPlans + bestCPUPlans[1] = h.getFragmentCPUPlans(h.fragmentCores, fragment) + bestCapacity = capacity + } + } + + cpuPlans := []types.CPUMap{} + for i := 0; i < bestCapacity; i++ { + fullCPUPlans := bestCPUPlans[0] + fragmentCPUPlans := bestCPUPlans[1] + + cpuMap := types.CPUMap{} + cpuMap.Add(fullCPUPlans[i]) + cpuMap.Add(fragmentCPUPlans[i]) + + cpuPlans = append(cpuPlans, cpuMap) + } + + return cpuPlans +} + +func (h *host) getFullCPUPlans(cores []*cpuCore, full int) []types.CPUMap { + if h.affinity { + return h.getFullCPUPlansWithAffinity(cores, full) + } + + result := []types.CPUMap{} + cpuHeap := &cpuCoreHeap{} + indexMap := map[string]int{} + for i, core := range cores { + indexMap[core.id] = i + cpuHeap.Push(&cpuCore{id: core.id, pieces: core.pieces}) + } + heap.Init(cpuHeap) + + for cpuHeap.Len() >= full { + plan := types.CPUMap{} + resourcesToPush := []*cpuCore{} + + for i := 0; i < full; i++ { + core := heap.Pop(cpuHeap).(*cpuCore) + plan[core.id] = h.shareBase + + core.pieces -= h.shareBase + if core.pieces > 0 { + resourcesToPush = append(resourcesToPush, core) + } + } + + result = append(result, plan) + for _, core := range resourcesToPush { + heap.Push(cpuHeap, core) + } + } + + // Try to ensure the effectiveness of the previous priority + sumOfIds := func(c types.CPUMap) int { + sum := 0 + for id := range c { + sum += indexMap[id] + } + return sum + } + + sort.Slice(result, func(i, j int) bool { return sumOfIds(result[i]) < sumOfIds(result[j]) }) + + return result +} + +func (h *host) getFullCPUPlansWithAffinity(cores []*cpuCore, full int) []types.CPUMap { + result := []types.CPUMap{} + + for len(cores) >= full { + count := len(cores) / full + tempCores := []*cpuCore{} + for i := 0; i < count; i++ { + cpuMap := types.CPUMap{} + for j := i * full; j < i*full+full; j++ { + cpuMap[cores[j].id] = h.shareBase + + remainingPieces := cores[j].pieces - h.shareBase + if remainingPieces > 0 { + tempCores = append(tempCores, &cpuCore{id: cores[j].id, pieces: remainingPieces}) + } + } + result = append(result, cpuMap) + } + + cores = append(tempCores, cores[len(cores)/full*full:]...) + } + + return result +} + +func (h *host) getFragmentCPUPlans(cores []*cpuCore, fragment int) []types.CPUMap { + result := []types.CPUMap{} + for _, core := range cores { + for i := 0; i < core.pieces/fragment; i++ { + result = append(result, types.CPUMap{core.id: fragment}) + } + } + return result +} diff --git a/resources/cpumem/schedule/schedule_test.go b/resources/cpumem/schedule/schedule_test.go new file mode 100644 index 000000000..37fab716d --- /dev/null +++ b/resources/cpumem/schedule/schedule_test.go @@ -0,0 +1,516 @@ +package schedule + +import ( + "strconv" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/cpumem/types" +) + +func TestGetFullCPUPlans(t *testing.T) { + h := newHost(types.CPUMap{ + "0": 400, + "1": 200, + "2": 400, + }, 100, -1) + cpuPlans := h.getFullCPUPlans(h.fullCores, 2) + assert.Equal(t, 5, len(cpuPlans)) + assert.ElementsMatch(t, cpuPlans, []types.CPUMap{ + {"0": 100, "1": 100}, + {"0": 100, "2": 100}, + {"0": 100, "2": 100}, + {"0": 100, "2": 100}, + {"1": 100, "2": 100}, + }) + + h = newHost(types.CPUMap{ + "0": 200, + "1": 200, + "2": 200, + }, 100, -1) + cpuPlans = h.getFullCPUPlans(h.fullCores, 2) + assert.EqualValues(t, 3, len(cpuPlans)) + assert.ElementsMatch(t, cpuPlans, []types.CPUMap{ + {"0": 100, "1": 100}, + {"0": 100, "2": 100}, + {"1": 100, "2": 100}, + }) +} + +func TestGetCPUPlansWithAffinity(t *testing.T) { + // 1.7 -> 1.0 + cpuMap := types.CPUMap{ + "0": 0, + "1": 30, + "2": 0, + } + originCPUMap := types.CPUMap{ + "0": 100, + "1": 30, + "2": 40, + } + resourceInfo := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{CPUMap: cpuMap, CPU: float64(len(cpuMap))}, + Usage: &types.NodeResourceArgs{}, + } + resourceInfo.Capacity.CPUMap.Add(originCPUMap) + cpuPlans := GetCPUPlans(resourceInfo, originCPUMap, 100, -1, &types.WorkloadResourceOpts{CPUBind: true, CPURequest: 1}) + assert.Equal(t, 1, len(cpuPlans)) + assert.Equal(t, cpuPlans[0].CPUMap, types.CPUMap{"0": 100}) + + // 1.7 -> 1.2 + cpuMap = types.CPUMap{ + "0": 0, + "1": 30, + "2": 0, + } + originCPUMap = types.CPUMap{ + "0": 100, + "1": 30, + "2": 40, + } + + resourceInfo = &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{CPUMap: cpuMap, CPU: float64(len(cpuMap))}, + Usage: &types.NodeResourceArgs{}, + } + resourceInfo.Capacity.CPUMap.Add(originCPUMap) + cpuPlans = GetCPUPlans(resourceInfo, originCPUMap, 100, -1, &types.WorkloadResourceOpts{CPUBind: true, CPURequest: 1.2}) + assert.Equal(t, 1, len(cpuPlans)) + assert.Equal(t, cpuPlans[0].CPUMap, types.CPUMap{"0": 100, "1": 20}) + + // 1.7 -> 2 + cpuMap = types.CPUMap{ + "0": 0, + "1": 80, + "2": 0, + "3": 0, + } + originCPUMap = types.CPUMap{ + "0": 100, + "1": 20, + "2": 40, + "3": 10, + } + + resourceInfo = &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{CPUMap: cpuMap, CPU: float64(len(cpuMap))}, + Usage: &types.NodeResourceArgs{}, + } + resourceInfo.Capacity.CPUMap.Add(originCPUMap) + cpuPlans = GetCPUPlans(resourceInfo, originCPUMap, 100, -1, &types.WorkloadResourceOpts{CPUBind: true, CPURequest: 2}) + assert.Equal(t, 1, len(cpuPlans)) + assert.Equal(t, cpuPlans[0].CPUMap, types.CPUMap{"0": 100, "1": 100}) + + // 1.7 -> 2 without enough pieces + cpuMap = types.CPUMap{ + "0": 0, + "1": 69, + "2": 10, + } + originCPUMap = types.CPUMap{ + "0": 100, + "1": 30, + "2": 40, + } + + resourceInfo = &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{CPUMap: cpuMap, CPU: float64(len(cpuMap))}, + Usage: &types.NodeResourceArgs{}, + } + resourceInfo.Capacity.CPUMap.Add(originCPUMap) + cpuPlans = GetCPUPlans(resourceInfo, originCPUMap, 100, -1, &types.WorkloadResourceOpts{CPUBind: true, CPURequest: 2}) + assert.Equal(t, 0, len(cpuPlans)) + + // 1.7 -> 2 + cpuMap = types.CPUMap{ + "0": 0, + "1": 70, + "2": 10, + } + originCPUMap = types.CPUMap{ + "0": 100, + "1": 30, + "2": 40, + } + + resourceInfo = &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{CPUMap: cpuMap, CPU: float64(len(cpuMap))}, + Usage: &types.NodeResourceArgs{}, + } + resourceInfo.Capacity.CPUMap.Add(originCPUMap) + cpuPlans = GetCPUPlans(resourceInfo, originCPUMap, 100, -1, &types.WorkloadResourceOpts{CPUBind: true, CPURequest: 2}) + assert.Equal(t, 1, len(cpuPlans)) + assert.Equal(t, cpuPlans[0].CPUMap, types.CPUMap{"0": 100, "1": 100}) + + // 1.7 -> 2 + cpuMap = types.CPUMap{ + "0": 100, + "1": 60, + "2": 0, + "3": 100, + "4": 100, + } + originCPUMap = types.CPUMap{ + "0": 100, + "1": 30, + "2": 40, + } + + resourceInfo = &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{CPUMap: cpuMap, CPU: float64(len(cpuMap))}, + Usage: &types.NodeResourceArgs{}, + } + resourceInfo.Capacity.CPUMap.Add(originCPUMap) + cpuPlans = GetCPUPlans(resourceInfo, originCPUMap, 100, -1, &types.WorkloadResourceOpts{CPUBind: true, CPURequest: 2}) + assert.Equal(t, 2, len(cpuPlans)) + assert.ElementsMatch(t, cpuPlans, []*CPUPlan{ + {CPUMap: types.CPUMap{"0": 100, "3": 100}}, + {CPUMap: types.CPUMap{"0": 100, "4": 100}}, + }) + + // 1.7 -> 2 + cpuMap = types.CPUMap{ + "0": 0, + "1": 60, + "2": 0, + } + originCPUMap = types.CPUMap{ + "0": 100, + "1": 30, + "2": 40, + } + + resourceInfo = &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{CPUMap: cpuMap, CPU: float64(len(cpuMap))}, + Usage: &types.NodeResourceArgs{}, + } + resourceInfo.Capacity.CPUMap.Add(originCPUMap) + cpuPlans = GetCPUPlans(resourceInfo, originCPUMap, 100, -1, &types.WorkloadResourceOpts{CPUBind: true, CPURequest: 2}) + assert.Equal(t, 0, len(cpuPlans)) +} + +func TestCPUOverSell(t *testing.T) { + var cpuMap types.CPUMap + var resourceInfo *types.NodeResourceInfo + maxShare := -1 + shareBase := 100 + + // oversell + cpuMap = types.CPUMap{"0": 300, "1": 300} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans := GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 2, + MemRequest: 1, + }) + assert.Equal(t, len(cpuPlans), 3) + assert.ElementsMatch(t, cpuPlans, []*CPUPlan{ + {CPUMap: types.CPUMap{"0": 100, "1": 100}}, + {CPUMap: types.CPUMap{"0": 100, "1": 100}}, + {CPUMap: types.CPUMap{"0": 100, "1": 100}}, + }) + + // one core oversell + cpuMap = types.CPUMap{"0": 300} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 0.5, + MemRequest: 1, + }) + assert.Equal(t, len(cpuPlans), 6) + assert.ElementsMatch(t, cpuPlans, []*CPUPlan{ + {CPUMap: types.CPUMap{"0": 50}}, + {CPUMap: types.CPUMap{"0": 50}}, + {CPUMap: types.CPUMap{"0": 50}}, + {CPUMap: types.CPUMap{"0": 50}}, + {CPUMap: types.CPUMap{"0": 50}}, + {CPUMap: types.CPUMap{"0": 50}}, + }) + + // balance + cpuMap = types.CPUMap{"0": 100, "1": 200, "2": 300} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1, + MemRequest: 1, + }) + assert.Equal(t, len(cpuPlans), 6) + assert.ElementsMatch(t, cpuPlans[:2], []*CPUPlan{ + {CPUMap: types.CPUMap{"0": 100}}, + {CPUMap: types.CPUMap{"1": 100}}, + }) + + // complex + cpuMap = types.CPUMap{"0": 50, "1": 100, "2": 300, "3": 70, "4": 200, "5": 30, "6": 230} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.7, + MemRequest: 1, + }) + assert.True(t, len(cpuPlans) >= 2) + + cpuMap = types.CPUMap{"0": 70, "1": 100, "2": 400} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.3, + MemRequest: 1, + }) + assert.Equal(t, len(cpuPlans), 4) + assert.ElementsMatch(t, cpuPlans, []*CPUPlan{ + {CPUMap: types.CPUMap{"0": 30, "2": 100}}, + {CPUMap: types.CPUMap{"0": 30, "2": 100}}, + {CPUMap: types.CPUMap{"1": 30, "2": 100}}, + {CPUMap: types.CPUMap{"1": 30, "2": 100}}, + }) +} + +func applyCPUPlans(t *testing.T, resourceInfo *types.NodeResourceInfo, cpuPlans []*CPUPlan) { + for _, cpuPlan := range cpuPlans { + resourceInfo.Usage.CPUMap.Add(cpuPlan.CPUMap) + } + assert.Nil(t, resourceInfo.Validate()) +} + +func TestCPUOverSellAndStableFragmentCore(t *testing.T) { + var cpuMap types.CPUMap + var resourceInfo *types.NodeResourceInfo + var cpuPlans []*CPUPlan + maxShare := -1 + shareBase := 100 + + // oversell + cpuMap = types.CPUMap{"0": 300, "1": 300} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.7, + MemRequest: 1, + }) + assert.True(t, len(cpuPlans) > 0) + + // stable fragment core + cpuMap = types.CPUMap{"0": 230, "1": 200} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.7, + MemRequest: 1, + }) + assert.True(t, len(cpuPlans) > 0) + assert.Equal(t, cpuPlans[0].CPUMap, types.CPUMap{"0": 70, "1": 100}) + applyCPUPlans(t, resourceInfo, cpuPlans[:1]) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.7, + MemRequest: 1, + }) + assert.True(t, len(cpuPlans) > 0) + assert.Equal(t, cpuPlans[0].CPUMap, types.CPUMap{"0": 70, "1": 100}) + + // complex node + cpuMap = types.CPUMap{"0": 230, "1": 80, "2": 300, "3": 200} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.7, + MemRequest: 1, + }) + assert.True(t, len(cpuPlans) >= 2) + applyCPUPlans(t, resourceInfo, cpuPlans[:2]) + assert.Equal(t, resourceInfo.Usage.CPUMap, types.CPUMap{"0": 70, "1": 70, "2": 0, "3": 200}) + + // consume full core + cpuMap = types.CPUMap{"0": 70, "1": 50, "2": 100, "3": 100, "4": 100} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.7, + MemRequest: 1, + }) + assert.True(t, len(cpuPlans) >= 2) + applyCPUPlans(t, resourceInfo, cpuPlans[:2]) + assert.Equal(t, resourceInfo.Usage.CPUMap, types.CPUMap{"0": 70, "1": 0, "2": 70, "3": 100, "4": 100}) + + // consume less fragment core + cpuMap = types.CPUMap{"0": 70, "1": 50, "2": 90} + resourceInfo = &types.NodeResourceInfo{Capacity: &types.NodeResourceArgs{ + CPU: float64(len(cpuMap)), + CPUMap: cpuMap, + Memory: 12 * units.GiB, + }} + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 0.5, + MemRequest: 1, + }) + assert.True(t, len(cpuPlans) >= 2) + applyCPUPlans(t, resourceInfo, cpuPlans[:2]) + assert.Equal(t, resourceInfo.Usage.CPUMap, types.CPUMap{"0": 50, "1": 50, "2": 0}) +} + +func TestNUMANodes(t *testing.T) { + maxShare := -1 + shareBase := 100 + + // same numa node + resourceInfo := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + CPU: 4, + CPUMap: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100}, + Memory: 4 * units.GiB, + NUMAMemory: types.NUMAMemory{"0": 2 * units.GiB, "1": 2 * units.GiB}, + NUMA: types.NUMA{"0": "0", "1": "0", "2": "1", "3": "1"}, + }, + Usage: nil, + } + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans := GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.3, + MemRequest: 1, + }) + assert.Equal(t, 2, len(cpuPlans)) + assert.ElementsMatch(t, cpuPlans, []*CPUPlan{ + {CPUMap: types.CPUMap{"0": 30, "1": 100}, NUMANode: "0"}, + {CPUMap: types.CPUMap{"2": 30, "3": 100}, NUMANode: "1"}, + }) + + // same numa node + cross numa node + resourceInfo = &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + CPU: 4, + CPUMap: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100, "4": 100, "5": 100}, + Memory: 6 * units.GiB, + NUMAMemory: types.NUMAMemory{"0": 3 * units.GiB, "1": 3 * units.GiB}, + NUMA: types.NUMA{"0": "0", "1": "0", "2": "0", "3": "1", "4": "1", "5": "1"}, + }, + Usage: nil, + } + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans = GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 2, + MemRequest: 2 * units.GiB, + }) + assert.Equal(t, 3, len(cpuPlans)) + assert.ElementsMatch(t, cpuPlans, []*CPUPlan{ + {CPUMap: types.CPUMap{"1": 100, "2": 100}, NUMANode: "0"}, + {CPUMap: types.CPUMap{"4": 100, "5": 100}, NUMANode: "1"}, + {CPUMap: types.CPUMap{"0": 100, "3": 100}, NUMANode: ""}, + }) +} + +func TestInsufficientMemory(t *testing.T) { + maxShare := -1 + shareBase := 100 + + resourceInfo := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + CPU: 4, + CPUMap: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100}, + Memory: 4 * units.GiB, + }, + Usage: nil, + } + assert.Nil(t, resourceInfo.Validate()) + + cpuPlans := GetCPUPlans(resourceInfo, nil, shareBase, maxShare, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.3, + MemRequest: 3 * units.GiB, + }) + assert.Equal(t, 1, len(cpuPlans)) + assert.ElementsMatch(t, cpuPlans, []*CPUPlan{ + {CPUMap: types.CPUMap{"0": 30, "1": 100}}, + }) +} + +func BenchmarkGetCPUPlans(b *testing.B) { + b.StopTimer() + resourceInfo := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + CPU: 24, + CPUMap: types.CPUMap{}, + Memory: 128 * units.GiB, + }, + } + for i := 0; i < 24; i++ { + resourceInfo.Capacity.CPUMap[strconv.Itoa(i)] = 100 + } + assert.Nil(b, resourceInfo.Validate()) + b.StartTimer() + for i := 0; i < b.N; i++ { + assert.True(b, len(GetCPUPlans(resourceInfo, nil, 100, -1, &types.WorkloadResourceOpts{ + CPUBind: true, + CPURequest: 1.3, + MemRequest: 1, + })) > 0) + } +} diff --git a/resources/cpumem/types/errors.go b/resources/cpumem/types/errors.go new file mode 100644 index 000000000..52505f968 --- /dev/null +++ b/resources/cpumem/types/errors.go @@ -0,0 +1,19 @@ +package types + +import "errors" + +var ( + ErrInvalidCapacity = errors.New("invalid resource capacity") + ErrInvalidCPUMap = errors.New("invalid cpu map") + ErrInvalidNUMA = errors.New("invalid numa") + ErrInvalidNUMAMemory = errors.New("invalid numa memory") + ErrInvalidMemory = errors.New("invalid memory") + ErrInvalidCPU = errors.New("invalid cpu") + + ErrInsufficientCPU = errors.New("cannot alloc a plan, not enough cpu") + ErrInsufficientMem = errors.New("cannot alloc a plan, not enough memory") + ErrInsufficientResource = errors.New("cannot alloc a plan, not enough resource") + + ErrNodeExists = errors.New("node already exists") + ErrNoNode = errors.New("no node") +) diff --git a/resources/cpumem/types/resource.go b/resources/cpumem/types/resource.go new file mode 100644 index 000000000..17c51f86c --- /dev/null +++ b/resources/cpumem/types/resource.go @@ -0,0 +1,475 @@ +package types + +import ( + "encoding/json" + "fmt" + "strconv" + "strings" + + "github.com/pkg/errors" + + coretypes "github.com/projecteru2/core/types" + coreutils "github.com/projecteru2/core/utils" +) + +// CPUMap . +type CPUMap map[string]int + +// TotalPieces . +func (c CPUMap) TotalPieces() int { + res := 0 + for _, pieces := range c { + res += pieces + } + return res +} + +// Sub . +func (c CPUMap) Sub(c1 CPUMap) { + for cpu, pieces := range c1 { + c[cpu] -= pieces + } +} + +// Add . +func (c CPUMap) Add(c1 CPUMap) { + for cpu, pieces := range c1 { + c[cpu] += pieces + } +} + +// NUMA map[cpuID]nodeID +type NUMA map[string]string + +// NUMAMemory . +type NUMAMemory map[string]int64 + +// Add . +func (n NUMAMemory) Add(n1 NUMAMemory) { + for numaNodeID, memory := range n1 { + n[numaNodeID] += memory + } +} + +// Sub . +func (n NUMAMemory) Sub(n1 NUMAMemory) { + for numaNodeID, memory := range n1 { + n[numaNodeID] -= memory + } +} + +// WorkloadResourceArgs . +type WorkloadResourceArgs struct { + CPURequest float64 `json:"cpu_request"` + CPULimit float64 `json:"cpu_limit"` + MemoryRequest int64 `json:"memory_request"` + MemoryLimit int64 `json:"memory_limit"` + CPUMap CPUMap `json:"cpu_map"` + NUMAMemory NUMAMemory `json:"numa_memory"` + NUMANode string `json:"numa_node"` +} + +// ParseFromRawParams . +func (r *WorkloadResourceArgs) ParseFromRawParams(rawParams coretypes.RawParams) error { + body, err := json.Marshal(rawParams) + if err != nil { + return err + } + return json.Unmarshal(body, r) +} + +// DeepCopy . +func (r *WorkloadResourceArgs) DeepCopy() *WorkloadResourceArgs { + res := &WorkloadResourceArgs{ + CPURequest: r.CPURequest, + CPULimit: r.CPULimit, + MemoryRequest: r.MemoryRequest, + MemoryLimit: r.MemoryLimit, + CPUMap: CPUMap{}, + NUMAMemory: NUMAMemory{}, + NUMANode: r.NUMANode, + } + + for cpu, pieces := range r.CPUMap { + res.CPUMap[cpu] = pieces + } + for cpuID, numaNodeID := range res.NUMAMemory { + res.NUMAMemory[cpuID] = numaNodeID + } + + return res +} + +// Add . +func (r *WorkloadResourceArgs) Add(r1 *WorkloadResourceArgs) { + r.CPURequest = coreutils.Round(r.CPURequest + r1.CPURequest) + r.MemoryRequest += r1.MemoryRequest + r.CPUMap.Add(r1.CPUMap) + + if len(r.NUMAMemory) == 0 { + r.NUMAMemory = r1.NUMAMemory + } else { + r.NUMAMemory.Add(r1.NUMAMemory) + } +} + +// Sub . +func (r *WorkloadResourceArgs) Sub(r1 *WorkloadResourceArgs) { + r.CPURequest = coreutils.Round(r.CPURequest - r1.CPURequest) + r.MemoryRequest -= r1.MemoryRequest + r.CPUMap.Sub(r1.CPUMap) + if r.NUMAMemory == nil { + r.NUMAMemory = NUMAMemory{} + } + r.NUMAMemory.Sub(r1.NUMAMemory) +} + +// NodeResourceArgs . +type NodeResourceArgs struct { + CPU float64 `json:"cpu"` + CPUMap CPUMap `json:"cpu_map"` + Memory int64 `json:"memory"` + NUMAMemory NUMAMemory `json:"numa_memory"` + NUMA NUMA `json:"numa"` +} + +// ParseFromRawParams . +func (r *NodeResourceArgs) ParseFromRawParams(rawParams coretypes.RawParams) error { + body, err := json.Marshal(rawParams) + if err != nil { + return err + } + return json.Unmarshal(body, r) +} + +// DeepCopy . +func (r *NodeResourceArgs) DeepCopy() *NodeResourceArgs { + res := &NodeResourceArgs{ + CPU: r.CPU, + CPUMap: CPUMap{}, + Memory: r.Memory, + NUMAMemory: NUMAMemory{}, + NUMA: NUMA{}, + } + + for cpu := range r.CPUMap { + res.CPUMap[cpu] = r.CPUMap[cpu] + } + for numaNodeID := range r.NUMAMemory { + res.NUMAMemory[numaNodeID] = r.NUMAMemory[numaNodeID] + } + for cpuID := range r.NUMA { + res.NUMA[cpuID] = r.NUMA[cpuID] + } + return res +} + +// Add . +func (r *NodeResourceArgs) Add(r1 *NodeResourceArgs) { + r.CPU = coreutils.Round(r.CPU + r1.CPU) + r.CPUMap.Add(r1.CPUMap) + r.Memory += r1.Memory + + for numaNodeID := range r1.NUMAMemory { + r.NUMAMemory[numaNodeID] += r1.NUMAMemory[numaNodeID] + } + + if len(r1.NUMA) > 0 { + r.NUMA = r1.NUMA + } +} + +// Sub . +func (r *NodeResourceArgs) Sub(r1 *NodeResourceArgs) { + r.CPU = coreutils.Round(r.CPU - r1.CPU) + r.CPUMap.Sub(r1.CPUMap) + r.Memory -= r1.Memory + + for numaNodeID := range r1.NUMAMemory { + r.NUMAMemory[numaNodeID] -= r1.NUMAMemory[numaNodeID] + } +} + +// NodeResourceInfo . +type NodeResourceInfo struct { + Capacity *NodeResourceArgs `json:"capacity"` + Usage *NodeResourceArgs `json:"usage"` +} + +// DeepCopy . +func (n *NodeResourceInfo) DeepCopy() *NodeResourceInfo { + return &NodeResourceInfo{ + Capacity: n.Capacity.DeepCopy(), + Usage: n.Usage.DeepCopy(), + } +} + +// RemoveEmptyCores . +func (n *NodeResourceInfo) RemoveEmptyCores() { + keysToDelete := []string{} + for cpu := range n.Capacity.CPUMap { + if n.Capacity.CPUMap[cpu] == 0 && n.Usage.CPUMap[cpu] == 0 { + keysToDelete = append(keysToDelete, cpu) + } + } + for cpu := range n.Usage.CPUMap { + if n.Capacity.CPUMap[cpu] == 0 && n.Usage.CPUMap[cpu] == 0 { + keysToDelete = append(keysToDelete, cpu) + } + } + + for _, cpu := range keysToDelete { + delete(n.Capacity.CPUMap, cpu) + delete(n.Usage.CPUMap, cpu) + } + + n.Capacity.CPU = float64(len(n.Capacity.CPUMap)) +} + +func (n *NodeResourceInfo) Validate() error { + if n.Capacity == nil || len(n.Capacity.CPUMap) == 0 { + return ErrInvalidCapacity + } + if n.Usage == nil { + n.Usage = &NodeResourceArgs{ + CPU: 0, + CPUMap: CPUMap{}, + Memory: 0, + NUMAMemory: NUMAMemory{}, + } + for cpuID := range n.Capacity.CPUMap { + n.Usage.CPUMap[cpuID] = 0 + } + for numaNodeID := range n.Capacity.NUMAMemory { + n.Usage.NUMAMemory[numaNodeID] = 0 + } + } + if len(n.Capacity.CPUMap) == 0 { + return ErrInvalidCPUMap + } + + for cpu, piecesUsed := range n.Usage.CPUMap { + if totalPieces, ok := n.Capacity.CPUMap[cpu]; !ok || piecesUsed < 0 || totalPieces < 0 || piecesUsed > totalPieces { + return ErrInvalidCPUMap + } + } + + if len(n.Capacity.NUMA) > 0 { + for cpu := range n.Capacity.CPUMap { + if numaNodeID, ok := n.Capacity.NUMA[cpu]; !ok { + return ErrInvalidNUMA + } else if _, ok = n.Capacity.NUMAMemory[numaNodeID]; !ok { + return ErrInvalidNUMAMemory + } + } + + for numaNodeID, totalMemory := range n.Capacity.NUMAMemory { + if totalMemory < 0 { + return ErrInvalidNUMAMemory + } + if memoryUsed := n.Usage.NUMAMemory[numaNodeID]; memoryUsed < 0 || memoryUsed > totalMemory { + return ErrInvalidNUMAMemory + } + } + } + + // remove nil CPUMap / NUMA / NUMAMemory + n.Capacity = n.Capacity.DeepCopy() + n.Usage = n.Usage.DeepCopy() + + return nil +} + +func (n *NodeResourceInfo) GetAvailableResource() *NodeResourceArgs { + availableResourceArgs := n.Capacity.DeepCopy() + availableResourceArgs.Sub(n.Usage) + + return availableResourceArgs +} + +// WorkloadResourceOpts includes all possible fields passed by eru-core for editing workload +type WorkloadResourceOpts struct { + CPUBind bool `json:"cpu_bind"` + KeepCPUBind bool `json:"keep_cpu_bind"` + CPURequest float64 `json:"cpu_request"` + CPULimit float64 `json:"cpu_limit"` + MemRequest int64 `json:"mem_request"` + MemLimit int64 `json:"mem_limit"` +} + +// Validate . +func (w *WorkloadResourceOpts) Validate() error { + if w.CPURequest == 0 && w.CPULimit > 0 { + w.CPURequest = w.CPULimit + } + if w.MemLimit < 0 || w.MemRequest < 0 { + return errors.Wrap(ErrInvalidMemory, "limit or request less than 0") + } + if w.CPURequest < 0 || w.CPULimit < 0 { + return errors.Wrap(ErrInvalidCPU, "limit or request less than 0") + } + if w.CPURequest == 0 && w.CPUBind { + return errors.Wrap(ErrInvalidCPU, "unlimited request with bind") + } + if w.MemRequest == 0 && w.MemLimit > 0 { + w.MemRequest = w.MemLimit + } + if w.MemLimit > 0 && w.MemRequest > 0 && w.MemLimit < w.MemRequest { + w.MemLimit = w.MemRequest + } + if w.CPURequest > 0 && w.CPULimit > 0 && w.CPULimit < w.CPURequest { + w.CPULimit = w.CPURequest + } + // if CPUBind=true, set cpu request=limit to solve the dilemma + // only deal with cpu limit>request but not vice versa + if w.CPUBind && w.CPURequest > 0 && w.CPULimit > 0 && w.CPULimit > w.CPURequest { + w.CPURequest = w.CPULimit + } + return nil +} + +// ParseFromRawParams . +func (w *WorkloadResourceOpts) ParseFromRawParams(rawParams coretypes.RawParams) (err error) { + w.KeepCPUBind = rawParams.Bool("keep-cpu-bind") + w.CPUBind = rawParams.Bool("cpu-bind") + w.CPURequest = rawParams.Float64("cpu-request") + w.CPULimit = rawParams.Float64("cpu-limit") + // check if cpu shortcut is set + if cpu := rawParams.Float64("cpu"); cpu > 0 { + w.CPURequest = cpu + w.CPULimit = cpu + } + if w.MemRequest, err = coreutils.ParseRAMInHuman(rawParams.String("memory-request")); err != nil { + return err + } + if w.MemLimit, err = coreutils.ParseRAMInHuman(rawParams.String("memory-limit")); err != nil { + return err + } + // check if mem shortcut is set + if rawParams.IsSet("memory") { + var mem int64 + if mem, err = coreutils.ParseRAMInHuman(rawParams.String("memory")); err != nil { + return err + } + w.MemLimit = mem + w.MemRequest = mem + } + + return nil +} + +// NodeResourceOpts includes all possible fields passed by eru-core for editing node +type NodeResourceOpts struct { + CPUMap CPUMap `json:"cpu_map"` + Memory int64 `json:"memory"` + NUMA NUMA `json:"numa"` + NUMAMemory NUMAMemory `json:"numa_memory"` + + RawParams coretypes.RawParams `json:"-"` +} + +func (n *NodeResourceOpts) ParseFromRawParams(rawParams coretypes.RawParams) (err error) { + n.RawParams = rawParams + + if n.CPUMap == nil { + n.CPUMap = CPUMap{} + } + + if cpu := n.RawParams.Int64("cpu"); cpu > 0 { + share := n.RawParams.Int64("share") + if share == 0 { + share = 100 + } + + for i := int64(0); i < cpu; i++ { + n.CPUMap[fmt.Sprintf("%v", i)] = int(share) + } + } else if cpuList := n.RawParams.String("cpu"); cpuList != "" { + cpuMapList := strings.Split(cpuList, ",") + for _, cpus := range cpuMapList { + cpuConfigs := strings.Split(cpus, ":") + pieces, err := strconv.ParseInt(cpuConfigs[1], 10, 32) + if err != nil { + return err + } + cpuID := cpuConfigs[0] + if _, err := strconv.Atoi(cpuID); err != nil { + return err + } + n.CPUMap[cpuID] = int(pieces) + } + } + + if n.Memory, err = coreutils.ParseRAMInHuman(n.RawParams.String("memory")); err != nil { + return err + } + n.NUMA = NUMA{} + n.NUMAMemory = NUMAMemory{} + + for index, cpuList := range n.RawParams.StringSlice("numa-cpu") { + nodeID := fmt.Sprintf("%d", index) + for _, cpuID := range strings.Split(cpuList, ",") { + n.NUMA[cpuID] = nodeID + } + } + + for index, memoryStr := range n.RawParams.StringSlice("numa-memory") { + nodeID := fmt.Sprintf("%d", index) + mem, err := coreutils.ParseRAMInHuman(memoryStr) + if err != nil { + return err + } + n.NUMAMemory[nodeID] = mem + } + + return nil +} + +// SkipEmpty used for setting node resource capacity in absolute mode +func (n *NodeResourceOpts) SkipEmpty(resourceCapacity *NodeResourceArgs) { + if n == nil { + return + } + if !n.RawParams.IsSet("cpu") { + n.CPUMap = resourceCapacity.CPUMap + } + if !n.RawParams.IsSet("memory") { + n.Memory = resourceCapacity.Memory + } + if !n.RawParams.IsSet("numa-cpu") { + n.NUMA = resourceCapacity.NUMA + } + if !n.RawParams.IsSet("numa-memory") { + n.NUMAMemory = resourceCapacity.NUMAMemory + } +} + +// NodeCapacityInfo . +type NodeCapacityInfo struct { + Node string `json:"node"` + Capacity int `json:"capacity"` + Usage float64 `json:"usage"` + Rate float64 `json:"rate"` + Weight int `json:"weight"` +} + +// EngineArgs . +type EngineArgs struct { + CPU float64 `json:"cpu"` + CPUMap CPUMap `json:"cpu_map"` + NUMANode string `json:"numa_node"` + Memory int64 `json:"memory"` + Remap bool `json:"remap"` +} + +// WorkloadResourceArgsMap . +type WorkloadResourceArgsMap map[string]*WorkloadResourceArgs + +// ParseFromRawParamsMap . +func (w *WorkloadResourceArgsMap) ParseFromRawParamsMap(rawParamsMap map[string]coretypes.RawParams) error { + body, err := json.Marshal(rawParamsMap) + if err != nil { + return err + } + return json.Unmarshal(body, w) +} diff --git a/resources/helper.go b/resources/helper.go new file mode 100644 index 000000000..f64b84bf6 --- /dev/null +++ b/resources/helper.go @@ -0,0 +1,12 @@ +package resources + +import "encoding/json" + +// ToResp converts a map to a response +func ToResp(m interface{}, resp interface{}) error { + body, err := json.Marshal(m) + if err != nil { + return err + } + return json.Unmarshal(body, resp) +} diff --git a/resources/manager.go b/resources/manager.go new file mode 100644 index 000000000..648735d39 --- /dev/null +++ b/resources/manager.go @@ -0,0 +1,686 @@ +package resources + +import ( + "context" + "math" + "sync" + + "github.com/pkg/errors" + + enginetypes "github.com/projecteru2/core/engine/types" + "github.com/projecteru2/core/log" + "github.com/projecteru2/core/types" + "github.com/projecteru2/core/utils" +) + +// PluginManager manages plugins +type PluginManager struct { + config types.Config + plugins []Plugin +} + +// NewPluginManager creates a plugin manager +func NewPluginManager(config types.Config) (*PluginManager, error) { + pm := &PluginManager{ + config: config, + plugins: []Plugin{}, + } + + return pm, nil +} + +// LoadPlugins . +func (pm *PluginManager) LoadPlugins(ctx context.Context) { + pm.plugins = []Plugin{} + if len(pm.config.ResourcePluginsDir) > 0 { + pluginFiles, err := utils.ListAllExecutableFiles(pm.config.ResourcePluginsDir) + if err != nil { + log.Errorf(ctx, "[LoadPlugins] failed to list all executable files dir: %v, err: %v", pm.config.ResourcePluginsDir, err) + return + } + + for _, file := range pluginFiles { + log.Infof(ctx, "[LoadPlugins] load binary plugin: %v", file) + pm.plugins = append(pm.plugins, &BinaryPlugin{path: file, config: pm.config}) + } + } +} + +// AddPlugins adds a plugin (for test and debug) +func (pm *PluginManager) AddPlugins(plugins ...Plugin) { + pm.plugins = append(pm.plugins, plugins...) +} + +// GetPlugins is used for mock +func (pm *PluginManager) GetPlugins() []Plugin { + return pm.plugins +} + +func (pm *PluginManager) callPlugins(plugins []Plugin, f func(Plugin)) { + wg := &sync.WaitGroup{} + wg.Add(len(plugins)) + + for _, plugin := range plugins { + go func(p Plugin) { + defer wg.Done() + f(p) + }(plugin) + } + wg.Wait() +} + +func callPlugins[T any](ctx context.Context, plugins []Plugin, f func(Plugin) (T, error)) (map[Plugin]T, error) { + resMap := sync.Map{} + combinedErr := types.NewCombinedErr() + wg := &sync.WaitGroup{} + wg.Add(len(plugins)) + + for _, plugin := range plugins { + go func(p Plugin) { + defer wg.Done() + if res, err := f(p); err != nil { + log.Errorf(ctx, "[callPlugins] failed to call plugin %v, err: %v", p.Name(), err) + combinedErr.Append(p.Name(), err) + } else { + resMap.Store(p, res) + } + }(plugin) + } + wg.Wait() + + results := map[Plugin]T{} + resMap.Range(func(key, value interface{}) bool { + plugin := key.(Plugin) + res := value.(T) + results[plugin] = res + return true + }) + if len(results) == len(plugins) { + return results, nil + } + + return results, combinedErr +} + +func (pm *PluginManager) mergeNodeCapacityInfo(m1 map[string]*NodeCapacityInfo, m2 map[string]*NodeCapacityInfo) map[string]*NodeCapacityInfo { + if len(m1) == 0 { + return m2 + } + + res := map[string]*NodeCapacityInfo{} + for node, info1 := range m1 { + // all the capacities should > 0 + if info2, ok := m2[node]; ok { + res[node] = &NodeCapacityInfo{ + NodeName: node, + Capacity: utils.Min(info1.Capacity, info2.Capacity), + Rate: info1.Rate + info2.Rate*info2.Weight, + Usage: info1.Usage + info2.Usage*info2.Weight, + Weight: info1.Weight + info2.Weight, + } + } + } + return res +} + +// GetNodesDeployCapacity returns available nodes which meet all the requirements +// the caller should require locks +// pure calculation +func (pm *PluginManager) GetNodesDeployCapacity(ctx context.Context, nodeNames []string, resourceOpts types.WorkloadResourceOpts) (map[string]*NodeCapacityInfo, int, error) { + res := map[string]*NodeCapacityInfo{} + + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*GetNodesDeployCapacityResponse, error) { + resp, err := plugin.GetNodesDeployCapacity(ctx, nodeNames, resourceOpts) + if err != nil { + log.Errorf(ctx, "[GetNodesDeployCapacity] plugin %v failed to get available nodeNames, request %v, err %v", plugin.Name(), resourceOpts, err) + } + return resp, err + }) + + if err != nil { + return nil, 0, err + } + + // get nodeNames with all resource capacities > 0 + for _, infoMap := range respMap { + res = pm.mergeNodeCapacityInfo(res, infoMap.Nodes) + } + + total := 0 + + // weighted average + for _, info := range res { + info.Rate /= info.Weight + info.Usage /= info.Weight + if info.Capacity == math.MaxInt64 { + total = math.MaxInt64 + } else { + total += info.Capacity + } + } + + return res, total, nil +} + +// mergeEngineArgs e.g. {"file": ["/bin/sh:/bin/sh"], "cpu": 1.2, "cpu-bind": true} + {"file": ["/bin/ls:/bin/ls"], "mem": "1PB"} +// => {"file": ["/bin/sh:/bin/sh", "/bin/ls:/bin/ls"], "cpu": 1.2, "cpu-bind": true, "mem": "1PB"} +func (pm *PluginManager) mergeEngineArgs(ctx context.Context, m1 types.EngineArgs, m2 types.EngineArgs) (types.EngineArgs, error) { + res := types.EngineArgs{} + for key, value := range m1 { + res[key] = value + } + for key, value := range m2 { + if _, ok := res[key]; ok { + // only two string slices can be merged + _, ok1 := res[key].([]string) + _, ok2 := value.([]string) + if !ok1 || !ok2 { + log.Errorf(ctx, "[mergeEngineArgs] only two string slices can be merged! error key %v, m1[key] = %v, m2[key] = %v", key, m1[key], m2[key]) + return nil, types.ErrInvalidEngineArgs + } + res[key] = append(res[key].([]string), value.([]string)...) + } else { + res[key] = value + } + } + return res, nil +} + +// Alloc . +func (pm *PluginManager) Alloc(ctx context.Context, nodeName string, deployCount int, resourceOpts types.WorkloadResourceOpts) ([]types.EngineArgs, []map[string]types.WorkloadResourceArgs, error) { + resEngineArgs := make([]types.EngineArgs, deployCount) + resResourceArgs := make([]map[string]types.WorkloadResourceArgs, deployCount) + + // init engine args + for i := 0; i < deployCount; i++ { + resEngineArgs[i] = types.EngineArgs{} + resResourceArgs[i] = map[string]types.WorkloadResourceArgs{} + } + + return resEngineArgs, resResourceArgs, utils.PCR(ctx, + // prepare: calculate engine args and resource args + func(ctx context.Context) error { + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*GetDeployArgsResponse, error) { + resp, err := plugin.GetDeployArgs(ctx, nodeName, deployCount, resourceOpts) + if err != nil { + log.Errorf(ctx, "[Alloc] plugin %v failed to compute alloc args, request %v, node %v, deploy count %v, err %v", plugin.Name(), resourceOpts, nodeName, deployCount, err) + } + return resp, err + }) + if err != nil { + return err + } + + // calculate engine args + for plugin, resp := range respMap { + for index, args := range resp.ResourceArgs { + resResourceArgs[index][plugin.Name()] = args + } + for index, args := range resp.EngineArgs { + resEngineArgs[index], err = pm.mergeEngineArgs(ctx, resEngineArgs[index], args) + if err != nil { + log.Errorf(ctx, "[Alloc] invalid engine args") + return err + } + } + } + return nil + }, + // commit: update node resources + func(ctx context.Context) error { + if _, _, err := pm.SetNodeResourceUsage(ctx, nodeName, nil, nil, resResourceArgs, true, Incr); err != nil { + log.Errorf(ctx, "[Alloc] failed to update node resource, err: %v", err) + return err + } + return nil + }, + // rollback: do nothing + func(ctx context.Context) error { + return nil + }, + pm.config.GlobalTimeout, + ) +} + +// RollbackAlloc rollbacks the allocated resource +func (pm *PluginManager) RollbackAlloc(ctx context.Context, nodeName string, resourceArgs []map[string]types.WorkloadResourceArgs) error { + _, _, err := pm.SetNodeResourceUsage(ctx, nodeName, nil, nil, resourceArgs, true, Decr) + return err +} + +// Realloc reallocates resource for workloads, returns engine args and final resource args. +func (pm *PluginManager) Realloc(ctx context.Context, nodeName string, originResourceArgs map[string]types.WorkloadResourceArgs, resourceOpts types.WorkloadResourceOpts) (types.EngineArgs, map[string]types.WorkloadResourceArgs, map[string]types.WorkloadResourceArgs, error) { + resEngineArgs := types.EngineArgs{} + resDeltaResourceArgs := map[string]types.WorkloadResourceArgs{} + resFinalResourceArgs := map[string]types.WorkloadResourceArgs{} + + return resEngineArgs, resDeltaResourceArgs, resFinalResourceArgs, utils.PCR(ctx, + // prepare: calculate engine args, delta node resource args and final workload resource args + func(ctx context.Context) error { + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*GetReallocArgsResponse, error) { + resp, err := plugin.GetReallocArgs(ctx, nodeName, originResourceArgs[plugin.Name()], resourceOpts) + if err != nil { + log.Errorf(ctx, "[Realloc] plugin %v failed to calculate realloc args, err: %v", plugin.Name(), err) + } + return resp, err + }) + + if err != nil { + log.Errorf(ctx, "[Realloc] realloc failed, origin: %+v, opts: %+v", originResourceArgs, resourceOpts) + return err + } + + for plugin, resp := range respMap { + if resEngineArgs, err = pm.mergeEngineArgs(ctx, resEngineArgs, resp.EngineArgs); err != nil { + log.Errorf(ctx, "[Realloc] invalid engine args, err: %v", err) + return err + } + resDeltaResourceArgs[plugin.Name()] = resp.Delta + resFinalResourceArgs[plugin.Name()] = resp.ResourceArgs + } + return nil + }, + // commit: update node resource + func(ctx context.Context) error { + if _, _, err := pm.SetNodeResourceUsage(ctx, nodeName, nil, nil, []map[string]types.WorkloadResourceArgs{resDeltaResourceArgs}, true, Incr); err != nil { + log.Errorf(ctx, "[Realloc] failed to update nodeName resource, err: %v", err) + return err + } + return nil + }, + // rollback: do nothing + func(ctx context.Context) error { + return nil + }, + pm.config.GlobalTimeout, + ) +} + +// RollbackRealloc rollbacks the resource changes caused by realloc +func (pm *PluginManager) RollbackRealloc(ctx context.Context, nodeName string, resourceArgs map[string]types.WorkloadResourceArgs) error { + _, _, err := pm.SetNodeResourceUsage(ctx, nodeName, nil, nil, []map[string]types.WorkloadResourceArgs{resourceArgs}, true, Decr) + return err +} + +// GetNodeResourceInfo . +func (pm *PluginManager) GetNodeResourceInfo(ctx context.Context, nodeName string, workloads []*types.Workload, fix bool) (map[string]types.NodeResourceArgs, map[string]types.NodeResourceArgs, []string, error) { + resResourceCapacity := map[string]types.NodeResourceArgs{} + resResourceUsage := map[string]types.NodeResourceArgs{} + resDiffs := []string{} + + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*GetNodeResourceInfoResponse, error) { + var resp *GetNodeResourceInfoResponse + var err error + if fix { + resp, err = plugin.FixNodeResource(ctx, nodeName, workloads) + } else { + resp, err = plugin.GetNodeResourceInfo(ctx, nodeName, workloads) + } + if err != nil { + log.Errorf(ctx, "[GetNodeResourceInfo] plugin %v failed to get node resource of node %v, err: %v", plugin.Name(), nodeName, err) + } + return resp, err + }) + + if err != nil { + return nil, nil, nil, err + } + + for plugin, resp := range respMap { + resResourceCapacity[plugin.Name()] = resp.ResourceInfo.Capacity + resResourceUsage[plugin.Name()] = resp.ResourceInfo.Usage + resDiffs = append(resDiffs, resp.Diffs...) + } + + return resResourceCapacity, resResourceUsage, resDiffs, nil +} + +// SetNodeResourceUsage with rollback +func (pm *PluginManager) SetNodeResourceUsage(ctx context.Context, nodeName string, nodeResourceOpts types.NodeResourceOpts, nodeResourceArgs map[string]types.NodeResourceArgs, workloadResourceArgs []map[string]types.WorkloadResourceArgs, delta bool, incr bool) (map[string]types.NodeResourceArgs, map[string]types.NodeResourceArgs, error) { + workloadResourceArgsMap := map[string][]types.WorkloadResourceArgs{} + rollbackPlugins := []Plugin{} + beforeMap := map[string]types.NodeResourceArgs{} + afterMap := map[string]types.NodeResourceArgs{} + + return beforeMap, afterMap, utils.PCR(ctx, + // prepare: convert []map[plugin]resourceArgs to map[plugin][]resourceArgs + // [{"cpu-plugin": {"cpu": 1}}, {"cpu-plugin": {"cpu": 1}}] -> {"cpu-plugin": [{"cpu": 1}, {"cpu": 1}]} + func(ctx context.Context) error { + for _, workloadResourceArgs := range workloadResourceArgs { + for plugin, rawParams := range workloadResourceArgs { + if _, ok := workloadResourceArgsMap[plugin]; !ok { + workloadResourceArgsMap[plugin] = []types.WorkloadResourceArgs{} + } + workloadResourceArgsMap[plugin] = append(workloadResourceArgsMap[plugin], rawParams) + } + } + if nodeResourceArgs == nil { + nodeResourceArgs = map[string]types.NodeResourceArgs{} + } + return nil + }, + // commit: call plugins to set node resource + func(ctx context.Context) error { + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*SetNodeResourceUsageResponse, error) { + resp, err := plugin.SetNodeResourceUsage(ctx, nodeName, nodeResourceOpts, nodeResourceArgs[plugin.Name()], workloadResourceArgsMap[plugin.Name()], delta, incr) + if err != nil { + log.Errorf(ctx, "[SetNodeResourceUsage] node %v plugin %v failed to update node resource, err: %v", nodeName, plugin.Name(), err) + } + return resp, err + }) + + if err != nil { + for plugin, resp := range respMap { + rollbackPlugins = append(rollbackPlugins, plugin) + beforeMap[plugin.Name()] = resp.Before + afterMap[plugin.Name()] = resp.After + } + + log.Errorf(ctx, "[UpdateNodeResourceUsage] failed to set node resource for node %v", nodeName) + return err + } + return nil + }, + // rollback: set the rollback resource args in reverse + func(ctx context.Context) error { + _, err := callPlugins(ctx, rollbackPlugins, func(plugin Plugin) (*SetNodeResourceUsageResponse, error) { + resp, err := plugin.SetNodeResourceUsage(ctx, nodeName, nil, beforeMap[plugin.Name()], nil, false, false) + if err != nil { + log.Errorf(ctx, "[UpdateNodeResourceUsage] node %v plugin %v failed to rollback node resource, err: %v", err) + } + return resp, err + }) + + if err != nil { + return err + } + return nil + }, + pm.config.GlobalTimeout, + ) +} + +// SetNodeResourceCapacity updates node resource capacity +// receives resource options instead of resource args +func (pm *PluginManager) SetNodeResourceCapacity(ctx context.Context, nodeName string, nodeResourceOpts types.NodeResourceOpts, nodeResourceArgs map[string]types.NodeResourceArgs, delta bool, incr bool) (map[string]types.NodeResourceArgs, map[string]types.NodeResourceArgs, error) { + rollbackPlugins := []Plugin{} + beforeMap := map[string]types.NodeResourceArgs{} + afterMap := map[string]types.NodeResourceArgs{} + + return beforeMap, afterMap, utils.PCR(ctx, + func(ctx context.Context) error { + if nodeResourceArgs == nil { + nodeResourceArgs = map[string]types.NodeResourceArgs{} + } + return nil + }, + // commit: call plugins to set node resource + func(ctx context.Context) error { + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*SetNodeResourceCapacityResponse, error) { + resp, err := plugin.SetNodeResourceCapacity(ctx, nodeName, nodeResourceOpts, nodeResourceArgs[plugin.Name()], delta, incr) + if err != nil { + log.Errorf(ctx, "[SetNodeResourceCapacity] node %v plugin %v failed to set node resource capacity, err: %v", nodeName, plugin.Name(), err) + } + return resp, err + }) + + if err != nil { + for plugin, resp := range respMap { + rollbackPlugins = append(rollbackPlugins, plugin) + beforeMap[plugin.Name()] = resp.Before + afterMap[plugin.Name()] = resp.After + } + + log.Errorf(ctx, "[SetNodeResourceCapacity] failed to set node resource for node %v", nodeName) + return err + } + return nil + }, + // rollback: set the rollback resource args in reverse + func(ctx context.Context) error { + _, err := callPlugins(ctx, rollbackPlugins, func(plugin Plugin) (*SetNodeResourceCapacityResponse, error) { + resp, err := plugin.SetNodeResourceCapacity(ctx, nodeName, nil, beforeMap[plugin.Name()], false, false) + if err != nil { + log.Errorf(ctx, "[SetNodeResourceCapacity] node %v plugin %v failed to rollback node resource capacity, err: %v", err) + } + return resp, err + }) + + if err != nil { + return err + } + return nil + }, + pm.config.GlobalTimeout, + ) +} + +// GetRemapArgs remaps resource and returns engine args for workloads. format: {"workload-1": {"cpus": ["1-3"]}} +// remap doesn't change resource args +func (pm *PluginManager) GetRemapArgs(ctx context.Context, nodeName string, workloadMap map[string]*types.Workload) (map[string]types.EngineArgs, error) { + resEngineArgsMap := map[string]types.EngineArgs{} + + // call plugins to remap + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*GetRemapArgsResponse, error) { + resp, err := plugin.GetRemapArgs(ctx, nodeName, workloadMap) + if err != nil { + log.Errorf(ctx, "[GetRemapArgs] plugin %v node %v failed to remap, err: %v", plugin.Name(), nodeName, err) + } + return resp, err + }) + + if err != nil { + return nil, err + } + + // merge engine args + for _, resp := range respMap { + for workloadID, engineArgs := range resp.EngineArgsMap { + if _, ok := resEngineArgsMap[workloadID]; !ok { + resEngineArgsMap[workloadID] = types.EngineArgs{} + } + resEngineArgsMap[workloadID], err = pm.mergeEngineArgs(ctx, resEngineArgsMap[workloadID], engineArgs) + if err != nil { + log.Errorf(ctx, "[GetRemapArgs] invalid engine args") + return nil, err + } + } + } + + return resEngineArgsMap, nil +} + +// AddNode . +func (pm *PluginManager) AddNode(ctx context.Context, nodeName string, resourceOpts types.NodeResourceOpts, nodeInfo *enginetypes.Info) (map[string]types.NodeResourceArgs, map[string]types.NodeResourceArgs, error) { + resResourceCapacity := map[string]types.NodeResourceArgs{} + resResourceUsage := map[string]types.NodeResourceArgs{} + rollbackPlugins := []Plugin{} + + return resResourceCapacity, resResourceUsage, utils.PCR(ctx, + // prepare: do nothing + func(ctx context.Context) error { + return nil + }, + // commit: call plugins to add the node + func(ctx context.Context) error { + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*AddNodeResponse, error) { + resp, err := plugin.AddNode(ctx, nodeName, resourceOpts, nodeInfo) + if err != nil { + log.Errorf(ctx, "[AddNode] node %v plugin %v failed to add node, req: %v, err: %v", nodeName, plugin.Name(), resourceOpts, err) + } + return resp, err + }) + + if err != nil { + for plugin := range respMap { + rollbackPlugins = append(rollbackPlugins, plugin) + } + + log.Errorf(ctx, "[AddNode] node %v failed to add node %v, rollback", nodeName, resourceOpts) + return err + } + + for plugin, resp := range respMap { + resResourceCapacity[plugin.Name()] = resp.Capacity + resResourceUsage[plugin.Name()] = resp.Usage + } + + return nil + }, + // rollback: remove node + func(ctx context.Context) error { + _, err := callPlugins(ctx, rollbackPlugins, func(plugin Plugin) (*RemoveNodeResponse, error) { + resp, err := plugin.RemoveNode(ctx, nodeName) + if err != nil { + log.Errorf(ctx, "[AddNode] node %v plugin %v failed to rollback, err: %v", nodeName, plugin.Name(), err) + } + return resp, err + }) + + if err != nil { + log.Errorf(ctx, "[AddNode] failed to rollback") + return err + } + + return nil + }, + pm.config.GlobalTimeout, + ) +} + +// RemoveNode . +func (pm *PluginManager) RemoveNode(ctx context.Context, nodeName string) error { + var resourceCapacityMap map[string]types.NodeResourceArgs + var resourceUsageMap map[string]types.NodeResourceArgs + rollbackPlugins := []Plugin{} + + return utils.PCR(ctx, + // prepare: get node resource + func(ctx context.Context) error { + var err error + resourceCapacityMap, resourceUsageMap, _, err = pm.GetNodeResourceInfo(ctx, nodeName, nil, false) + if err != nil { + log.Errorf(ctx, "[RemoveNode] failed to get node %v resource, err: %v", nodeName, err) + return err + } + return nil + }, + // commit: remove node + func(ctx context.Context) error { + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*RemoveNodeResponse, error) { + resp, err := plugin.RemoveNode(ctx, nodeName) + if err != nil { + log.Errorf(ctx, "[AddNode] plugin %v failed to remove node, err: %v", plugin.Name(), nodeName, err) + } + return resp, err + }) + + if err != nil { + for plugin := range respMap { + rollbackPlugins = append(rollbackPlugins, plugin) + } + + log.Errorf(ctx, "[AddNode] failed to remove node %v", nodeName) + return err + } + return nil + }, + // rollback: add node + func(ctx context.Context) error { + _, err := callPlugins(ctx, rollbackPlugins, func(plugin Plugin) (*SetNodeResourceInfoResponse, error) { + resp, err := plugin.SetNodeResourceInfo(ctx, nodeName, resourceCapacityMap[plugin.Name()], resourceUsageMap[plugin.Name()]) + if err != nil { + log.Errorf(ctx, "[RemoveNode] plugin %v node %v failed to rollback, err: %v", plugin.Name(), nodeName, err) + } + return resp, err + }) + + if err != nil { + log.Errorf(ctx, "[RemoveNode] failed to rollback") + return err + } + return nil + }, + pm.config.GlobalTimeout, + ) +} + +// GetMostIdleNode , +func (pm *PluginManager) GetMostIdleNode(ctx context.Context, nodeNames []string) (string, error) { + var mostIdleNode *GetMostIdleNodeResponse + + if len(nodeNames) == 0 { + return "", errors.Wrap(types.ErrGetMostIdleNodeFailed, "empty node names") + } + + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*GetMostIdleNodeResponse, error) { + resp, err := plugin.GetMostIdleNode(ctx, nodeNames) + if err != nil { + log.Errorf(ctx, "[GetMostIdleNode] plugin %v failed to get the most idle node of %v, err: %v", plugin.Name(), nodeNames, err) + } + return resp, err + }) + + if err != nil { + log.Errorf(ctx, "[GetMostIdleNode] failed to get the most idle node of %v", nodeNames) + return "", err + } + + for _, resp := range respMap { + if (mostIdleNode == nil || resp.Priority > mostIdleNode.Priority) && len(resp.NodeName) > 0 { + mostIdleNode = resp + } + } + + if mostIdleNode == nil { + return "", types.ErrGetMostIdleNodeFailed + } + return mostIdleNode.NodeName, nil +} + +// GetMetricsDescription . +func (pm *PluginManager) GetMetricsDescription(ctx context.Context) ([]*MetricsDescription, error) { + var metricsDescriptions []*MetricsDescription + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*GetMetricsDescriptionResponse, error) { + resp, err := plugin.GetMetricsDescription(ctx) + if err != nil { + log.Errorf(ctx, "[GetMetricsDescription] plugin %v failed to get metrics description, err: %v", plugin.Name(), err) + } + return resp, err + }) + + if err != nil { + log.Errorf(ctx, "[GetMetricsDescription] failed to get metrics description") + return nil, err + } + + for _, resp := range respMap { + metricsDescriptions = append(metricsDescriptions, *resp...) + } + + return metricsDescriptions, nil +} + +// ResolveNodeResourceInfoToMetrics . +func (pm *PluginManager) ResolveNodeResourceInfoToMetrics(ctx context.Context, podName string, nodeName string, nodeResourceCapacity map[string]types.NodeResourceArgs, nodeResourceUsage map[string]types.NodeResourceArgs) ([]*Metrics, error) { + var metrics []*Metrics + respMap, err := callPlugins(ctx, pm.plugins, func(plugin Plugin) (*ResolveNodeResourceInfoToMetricsResponse, error) { + capacity, usage := nodeResourceCapacity[plugin.Name()], nodeResourceUsage[plugin.Name()] + resp, err := plugin.ResolveNodeResourceInfoToMetrics(ctx, podName, nodeName, &NodeResourceInfo{Capacity: capacity, Usage: usage}) + if err != nil { + log.Errorf(ctx, "[ResolveNodeResourceInfoToMetrics] plugin %v failed to resolve node resource info to metrics, err: %v", plugin.Name(), err) + } + return resp, err + }) + + if err != nil { + log.Errorf(ctx, "[ResolveNodeResourceInfoToMetrics] failed to resolve node resource info to metrics") + return nil, err + } + + for _, resp := range respMap { + metrics = append(metrics, *resp...) + } + + return metrics, nil +} diff --git a/resources/mocks/Plugin.go b/resources/mocks/Plugin.go new file mode 100644 index 000000000..ae3dbcb09 --- /dev/null +++ b/resources/mocks/Plugin.go @@ -0,0 +1,355 @@ +// Code generated by mockery v0.0.0-dev. DO NOT EDIT. + +package mocks + +import ( + context "context" + + enginetypes "github.com/projecteru2/core/engine/types" + mock "github.com/stretchr/testify/mock" + + resources "github.com/projecteru2/core/resources" + + types "github.com/projecteru2/core/types" +) + +// Plugin is an autogenerated mock type for the Plugin type +type Plugin struct { + mock.Mock +} + +// AddNode provides a mock function with given fields: ctx, nodeName, resourceOpts, nodeInfo +func (_m *Plugin) AddNode(ctx context.Context, nodeName string, resourceOpts types.NodeResourceOpts, nodeInfo *enginetypes.Info) (*resources.AddNodeResponse, error) { + ret := _m.Called(ctx, nodeName, resourceOpts, nodeInfo) + + var r0 *resources.AddNodeResponse + if rf, ok := ret.Get(0).(func(context.Context, string, types.NodeResourceOpts, *enginetypes.Info) *resources.AddNodeResponse); ok { + r0 = rf(ctx, nodeName, resourceOpts, nodeInfo) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.AddNodeResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, types.NodeResourceOpts, *enginetypes.Info) error); ok { + r1 = rf(ctx, nodeName, resourceOpts, nodeInfo) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// FixNodeResource provides a mock function with given fields: ctx, nodeName, workloads +func (_m *Plugin) FixNodeResource(ctx context.Context, nodeName string, workloads []*types.Workload) (*resources.GetNodeResourceInfoResponse, error) { + ret := _m.Called(ctx, nodeName, workloads) + + var r0 *resources.GetNodeResourceInfoResponse + if rf, ok := ret.Get(0).(func(context.Context, string, []*types.Workload) *resources.GetNodeResourceInfoResponse); ok { + r0 = rf(ctx, nodeName, workloads) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.GetNodeResourceInfoResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, []*types.Workload) error); ok { + r1 = rf(ctx, nodeName, workloads) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetDeployArgs provides a mock function with given fields: ctx, nodeName, deployCount, resourceOpts +func (_m *Plugin) GetDeployArgs(ctx context.Context, nodeName string, deployCount int, resourceOpts types.WorkloadResourceOpts) (*resources.GetDeployArgsResponse, error) { + ret := _m.Called(ctx, nodeName, deployCount, resourceOpts) + + var r0 *resources.GetDeployArgsResponse + if rf, ok := ret.Get(0).(func(context.Context, string, int, types.WorkloadResourceOpts) *resources.GetDeployArgsResponse); ok { + r0 = rf(ctx, nodeName, deployCount, resourceOpts) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.GetDeployArgsResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, int, types.WorkloadResourceOpts) error); ok { + r1 = rf(ctx, nodeName, deployCount, resourceOpts) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetMetricsDescription provides a mock function with given fields: ctx +func (_m *Plugin) GetMetricsDescription(ctx context.Context) (*resources.GetMetricsDescriptionResponse, error) { + ret := _m.Called(ctx) + + var r0 *resources.GetMetricsDescriptionResponse + if rf, ok := ret.Get(0).(func(context.Context) *resources.GetMetricsDescriptionResponse); ok { + r0 = rf(ctx) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.GetMetricsDescriptionResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context) error); ok { + r1 = rf(ctx) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetMostIdleNode provides a mock function with given fields: ctx, nodeNames +func (_m *Plugin) GetMostIdleNode(ctx context.Context, nodeNames []string) (*resources.GetMostIdleNodeResponse, error) { + ret := _m.Called(ctx, nodeNames) + + var r0 *resources.GetMostIdleNodeResponse + if rf, ok := ret.Get(0).(func(context.Context, []string) *resources.GetMostIdleNodeResponse); ok { + r0 = rf(ctx, nodeNames) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.GetMostIdleNodeResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, []string) error); ok { + r1 = rf(ctx, nodeNames) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetNodeResourceInfo provides a mock function with given fields: ctx, nodeName, workloads +func (_m *Plugin) GetNodeResourceInfo(ctx context.Context, nodeName string, workloads []*types.Workload) (*resources.GetNodeResourceInfoResponse, error) { + ret := _m.Called(ctx, nodeName, workloads) + + var r0 *resources.GetNodeResourceInfoResponse + if rf, ok := ret.Get(0).(func(context.Context, string, []*types.Workload) *resources.GetNodeResourceInfoResponse); ok { + r0 = rf(ctx, nodeName, workloads) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.GetNodeResourceInfoResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, []*types.Workload) error); ok { + r1 = rf(ctx, nodeName, workloads) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetNodesDeployCapacity provides a mock function with given fields: ctx, nodeNames, resourceOpts +func (_m *Plugin) GetNodesDeployCapacity(ctx context.Context, nodeNames []string, resourceOpts types.WorkloadResourceOpts) (*resources.GetNodesDeployCapacityResponse, error) { + ret := _m.Called(ctx, nodeNames, resourceOpts) + + var r0 *resources.GetNodesDeployCapacityResponse + if rf, ok := ret.Get(0).(func(context.Context, []string, types.WorkloadResourceOpts) *resources.GetNodesDeployCapacityResponse); ok { + r0 = rf(ctx, nodeNames, resourceOpts) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.GetNodesDeployCapacityResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, []string, types.WorkloadResourceOpts) error); ok { + r1 = rf(ctx, nodeNames, resourceOpts) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetReallocArgs provides a mock function with given fields: ctx, nodeName, originResourceArgs, resourceOpts +func (_m *Plugin) GetReallocArgs(ctx context.Context, nodeName string, originResourceArgs types.WorkloadResourceArgs, resourceOpts types.WorkloadResourceOpts) (*resources.GetReallocArgsResponse, error) { + ret := _m.Called(ctx, nodeName, originResourceArgs, resourceOpts) + + var r0 *resources.GetReallocArgsResponse + if rf, ok := ret.Get(0).(func(context.Context, string, types.WorkloadResourceArgs, types.WorkloadResourceOpts) *resources.GetReallocArgsResponse); ok { + r0 = rf(ctx, nodeName, originResourceArgs, resourceOpts) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.GetReallocArgsResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, types.WorkloadResourceArgs, types.WorkloadResourceOpts) error); ok { + r1 = rf(ctx, nodeName, originResourceArgs, resourceOpts) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetRemapArgs provides a mock function with given fields: ctx, nodeName, workloadMap +func (_m *Plugin) GetRemapArgs(ctx context.Context, nodeName string, workloadMap map[string]*types.Workload) (*resources.GetRemapArgsResponse, error) { + ret := _m.Called(ctx, nodeName, workloadMap) + + var r0 *resources.GetRemapArgsResponse + if rf, ok := ret.Get(0).(func(context.Context, string, map[string]*types.Workload) *resources.GetRemapArgsResponse); ok { + r0 = rf(ctx, nodeName, workloadMap) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.GetRemapArgsResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, map[string]*types.Workload) error); ok { + r1 = rf(ctx, nodeName, workloadMap) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// Name provides a mock function with given fields: +func (_m *Plugin) Name() string { + ret := _m.Called() + + var r0 string + if rf, ok := ret.Get(0).(func() string); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(string) + } + + return r0 +} + +// RemoveNode provides a mock function with given fields: ctx, nodeName +func (_m *Plugin) RemoveNode(ctx context.Context, nodeName string) (*resources.RemoveNodeResponse, error) { + ret := _m.Called(ctx, nodeName) + + var r0 *resources.RemoveNodeResponse + if rf, ok := ret.Get(0).(func(context.Context, string) *resources.RemoveNodeResponse); ok { + r0 = rf(ctx, nodeName) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.RemoveNodeResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, nodeName) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// ResolveNodeResourceInfoToMetrics provides a mock function with given fields: ctx, podName, nodeName, nodeResourceInfo +func (_m *Plugin) ResolveNodeResourceInfoToMetrics(ctx context.Context, podName string, nodeName string, nodeResourceInfo *resources.NodeResourceInfo) (*resources.ResolveNodeResourceInfoToMetricsResponse, error) { + ret := _m.Called(ctx, podName, nodeName, nodeResourceInfo) + + var r0 *resources.ResolveNodeResourceInfoToMetricsResponse + if rf, ok := ret.Get(0).(func(context.Context, string, string, *resources.NodeResourceInfo) *resources.ResolveNodeResourceInfoToMetricsResponse); ok { + r0 = rf(ctx, podName, nodeName, nodeResourceInfo) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.ResolveNodeResourceInfoToMetricsResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, string, *resources.NodeResourceInfo) error); ok { + r1 = rf(ctx, podName, nodeName, nodeResourceInfo) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SetNodeResourceCapacity provides a mock function with given fields: ctx, nodeName, nodeResourceOpts, nodeResourceArgs, delta, incr +func (_m *Plugin) SetNodeResourceCapacity(ctx context.Context, nodeName string, nodeResourceOpts types.NodeResourceOpts, nodeResourceArgs types.NodeResourceArgs, delta bool, incr bool) (*resources.SetNodeResourceCapacityResponse, error) { + ret := _m.Called(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, delta, incr) + + var r0 *resources.SetNodeResourceCapacityResponse + if rf, ok := ret.Get(0).(func(context.Context, string, types.NodeResourceOpts, types.NodeResourceArgs, bool, bool) *resources.SetNodeResourceCapacityResponse); ok { + r0 = rf(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, delta, incr) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.SetNodeResourceCapacityResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, types.NodeResourceOpts, types.NodeResourceArgs, bool, bool) error); ok { + r1 = rf(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, delta, incr) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SetNodeResourceInfo provides a mock function with given fields: ctx, nodeName, resourceCapacity, resourceUsage +func (_m *Plugin) SetNodeResourceInfo(ctx context.Context, nodeName string, resourceCapacity types.NodeResourceArgs, resourceUsage types.NodeResourceArgs) (*resources.SetNodeResourceInfoResponse, error) { + ret := _m.Called(ctx, nodeName, resourceCapacity, resourceUsage) + + var r0 *resources.SetNodeResourceInfoResponse + if rf, ok := ret.Get(0).(func(context.Context, string, types.NodeResourceArgs, types.NodeResourceArgs) *resources.SetNodeResourceInfoResponse); ok { + r0 = rf(ctx, nodeName, resourceCapacity, resourceUsage) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.SetNodeResourceInfoResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, types.NodeResourceArgs, types.NodeResourceArgs) error); ok { + r1 = rf(ctx, nodeName, resourceCapacity, resourceUsage) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SetNodeResourceUsage provides a mock function with given fields: ctx, nodeName, nodeResourceOpts, nodeResourceArgs, workloadResourceArgs, delta, incr +func (_m *Plugin) SetNodeResourceUsage(ctx context.Context, nodeName string, nodeResourceOpts types.NodeResourceOpts, nodeResourceArgs types.NodeResourceArgs, workloadResourceArgs []types.WorkloadResourceArgs, delta bool, incr bool) (*resources.SetNodeResourceUsageResponse, error) { + ret := _m.Called(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, workloadResourceArgs, delta, incr) + + var r0 *resources.SetNodeResourceUsageResponse + if rf, ok := ret.Get(0).(func(context.Context, string, types.NodeResourceOpts, types.NodeResourceArgs, []types.WorkloadResourceArgs, bool, bool) *resources.SetNodeResourceUsageResponse); ok { + r0 = rf(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, workloadResourceArgs, delta, incr) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*resources.SetNodeResourceUsageResponse) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, types.NodeResourceOpts, types.NodeResourceArgs, []types.WorkloadResourceArgs, bool, bool) error); ok { + r1 = rf(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, workloadResourceArgs, delta, incr) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} diff --git a/resources/plugin.go b/resources/plugin.go new file mode 100644 index 000000000..ad2d2d440 --- /dev/null +++ b/resources/plugin.go @@ -0,0 +1,86 @@ +package resources + +import ( + "context" + + enginetypes "github.com/projecteru2/core/engine/types" + coretypes "github.com/projecteru2/core/types" +) + +const ( + // Incr increase + Incr = true + + // Decr decrease + Decr = false + + getNodesCapacityCommand = "get-capacity" + getNodeResourceInfoCommand = "get-node" + setNodeResourceInfoCommand = "set-node" + setNodeResourceUsageCommand = "set-node-usage" + setNodeResourceCapacityCommand = "set-node-capacity" + getDeployArgsCommand = "get-deploy-args" + getReallocArgsCommand = "get-realloc-args" + getRemapArgsCommand = "get-remap-args" + addNodeCommand = "add-node" + removeNodeCommand = "remove-node" + getMostIdleNodeCommand = "get-idle" + getMetricsDescriptionCommand = "desc-metrics" + resolveNodeResourceInfoToMetricsCommand = "resolve-metrics" +) + +type Plugin interface { + // GetDeployArgs tries to allocate resource, returns engine args for each workload, format: [{"cpus": 1.2}, {"cpus": 1.2}] + // also returns resource args for each workload, format: [{"cpus": 1.2}, {"cpus": 1.2}] + // pure calculation + GetDeployArgs(ctx context.Context, nodeName string, deployCount int, resourceOpts coretypes.WorkloadResourceOpts) (*GetDeployArgsResponse, error) + + // GetReallocArgs tries to reallocate resource, returns engine args, delta resource args and final resource args. + // should return error if resource of some node is not enough for the realloc operation. + // pure calculation + GetReallocArgs(ctx context.Context, nodeName string, originResourceArgs coretypes.WorkloadResourceArgs, resourceOpts coretypes.WorkloadResourceOpts) (*GetReallocArgsResponse, error) + + // GetRemapArgs tries to remap resources based on workload metadata and node resource usage, then returns engine args for workloads. + // pure calculation + GetRemapArgs(ctx context.Context, nodeName string, workloadMap map[string]*coretypes.Workload) (*GetRemapArgsResponse, error) + + // GetNodesDeployCapacity returns available nodes and total capacity + GetNodesDeployCapacity(ctx context.Context, nodeNames []string, resourceOpts coretypes.WorkloadResourceOpts) (*GetNodesDeployCapacityResponse, error) + + // GetMostIdleNode returns the most idle node for building + GetMostIdleNode(ctx context.Context, nodeNames []string) (*GetMostIdleNodeResponse, error) + + // GetNodeResourceInfo returns total resource info and available resource info of the node, format: {"cpu": 2} + // also returns diffs, format: ["node.VolumeUsed != sum(workload.VolumeRequest"] + GetNodeResourceInfo(ctx context.Context, nodeName string, workloads []*coretypes.Workload) (*GetNodeResourceInfoResponse, error) + + // FixNodeResource fixes the node resource usage by its workloads + FixNodeResource(ctx context.Context, nodeName string, workloads []*coretypes.Workload) (*GetNodeResourceInfoResponse, error) + + // SetNodeResourceUsage sets the amount of allocated resource info + SetNodeResourceUsage(ctx context.Context, nodeName string, nodeResourceOpts coretypes.NodeResourceOpts, nodeResourceArgs coretypes.NodeResourceArgs, workloadResourceArgs []coretypes.WorkloadResourceArgs, delta bool, incr bool) (*SetNodeResourceUsageResponse, error) + + // SetNodeResourceCapacity sets the amount of total resource info + SetNodeResourceCapacity(ctx context.Context, nodeName string, nodeResourceOpts coretypes.NodeResourceOpts, nodeResourceArgs coretypes.NodeResourceArgs, delta bool, incr bool) (*SetNodeResourceCapacityResponse, error) + + // SetNodeResourceInfo sets both total node resource info and allocated resource info + // used for rollback of RemoveNode + // notice: here uses absolute values, not delta values + SetNodeResourceInfo(ctx context.Context, nodeName string, resourceCapacity coretypes.NodeResourceArgs, resourceUsage coretypes.NodeResourceArgs) (*SetNodeResourceInfoResponse, error) + + // AddNode adds a node with requested resource, returns resource capacity and (empty) resource usage + // should return error if the node already exists + AddNode(ctx context.Context, nodeName string, resourceOpts coretypes.NodeResourceOpts, nodeInfo *enginetypes.Info) (*AddNodeResponse, error) + + // RemoveNode removes node + RemoveNode(ctx context.Context, nodeName string) (*RemoveNodeResponse, error) + + // GetMetricsDescription returns metrics description + GetMetricsDescription(ctx context.Context) (*GetMetricsDescriptionResponse, error) + + // ResolveNodeResourceInfoToMetrics resolves node resource info to metrics + ResolveNodeResourceInfoToMetrics(ctx context.Context, podName string, nodeName string, nodeResourceInfo *NodeResourceInfo) (*ResolveNodeResourceInfoToMetricsResponse, error) + + // Name returns the name of plugin + Name() string +} diff --git a/resources/resources.go b/resources/resources.go deleted file mode 100644 index 580a7583c..000000000 --- a/resources/resources.go +++ /dev/null @@ -1,25 +0,0 @@ -package resources - -import ( - "github.com/projecteru2/core/resources/cpumem" - "github.com/projecteru2/core/resources/storage" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/resources/volume" - "github.com/projecteru2/core/types" -) - -var registeredFactories = []func(types.ResourceOptions) (resourcetypes.ResourceRequest, error){ - cpumem.MakeRequest, - storage.MakeRequest, - volume.MakeRequest, -} - -// MakeRequests . -func MakeRequests(opts types.ResourceOptions) (resourceRequests resourcetypes.ResourceRequests, err error) { - for idx, factory := range registeredFactories { - if resourceRequests[idx], err = factory(opts); err != nil { - return - } - } - return -} diff --git a/resources/scheduler.go b/resources/scheduler.go deleted file mode 100644 index 7cb65a2bc..000000000 --- a/resources/scheduler.go +++ /dev/null @@ -1,36 +0,0 @@ -package resources - -import ( - "context" - - "github.com/projecteru2/core/log" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/types" -) - -// SelectNodesByResourceRequests select nodes by resource requests -func SelectNodesByResourceRequests(ctx context.Context, resourceRequests resourcetypes.ResourceRequests, nodeMap map[string]*types.Node) ( - plans []resourcetypes.ResourcePlans, - err error, -) { - scheduleInfos := []resourcetypes.ScheduleInfo{} - for _, node := range nodeMap { - nodeMeta, err := node.NodeMeta.DeepCopy() - if err != nil { - return nil, err - } - scheduleInfo := resourcetypes.ScheduleInfo{ - NodeMeta: nodeMeta, - } - scheduleInfos = append(scheduleInfos, scheduleInfo) - } - log.Debugf(ctx, "[SelectNodesByResourceRequests] scheduleInfos: %+v", scheduleInfos) - for _, resourceRequest := range resourceRequests { - plan, _, err := resourceRequest.MakeScheduler()(ctx, scheduleInfos) - if err != nil { - return plans, err - } - plans = append(plans, plan) - } - return -} diff --git a/resources/storage/storage.go b/resources/storage/storage.go deleted file mode 100644 index 0ee763dd0..000000000 --- a/resources/storage/storage.go +++ /dev/null @@ -1,121 +0,0 @@ -package storage - -import ( - "context" - - "github.com/pkg/errors" - - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - "github.com/projecteru2/core/types" - "github.com/projecteru2/core/utils" -) - -type storageRequest struct { - request int64 - limit int64 -} - -// MakeRequest . -func MakeRequest(opts types.ResourceOptions) (resourcetypes.ResourceRequest, error) { - sr := &storageRequest{ - request: opts.StorageRequest, - limit: opts.StorageLimit, - } - if sr.limit > 0 && sr.request == 0 { - sr.request = sr.limit - } - // add volume request / limit to storage request / limit - if len(opts.VolumeRequest) != 0 && len(opts.VolumeLimit) != len(opts.VolumeRequest) { - return nil, errors.Wrapf(types.ErrBadVolume, "volume request and limit must be the same length") - } - for idx := range opts.VolumeLimit { - if len(opts.VolumeRequest) > 0 { - sr.request += opts.VolumeRequest[idx].SizeInBytes - sr.limit += utils.Max(opts.VolumeLimit[idx].SizeInBytes, opts.VolumeRequest[idx].SizeInBytes) - } else { - sr.request += opts.VolumeLimit[idx].SizeInBytes - sr.limit += opts.VolumeLimit[idx].SizeInBytes - } - } - - return sr, sr.Validate() -} - -// Type . -func (s storageRequest) Type() types.ResourceType { - return types.ResourceStorage -} - -// Validate . -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 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 (s storageRequest) MakeScheduler() resourcetypes.SchedulerV2 { - return func(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo) (plans resourcetypes.ResourcePlans, total int, err error) { - schedulerV1, err := scheduler.GetSchedulerV1() - if err != nil { - return - } - - scheduleInfos, total, err = schedulerV1.SelectStorageNodes(ctx, scheduleInfos, s.request) - return ResourcePlans{ - request: s.request, - limit: s.limit, - capacity: resourcetypes.GetCapacity(scheduleInfos), - }, total, err - } -} - -// Rate . -func (s storageRequest) Rate(node types.Node) float64 { - return float64(s.request) / float64(node.InitStorageCap) -} - -// ResourcePlans . -type ResourcePlans struct { - request int64 - limit int64 - capacity map[string]int -} - -// Type . -func (rp ResourcePlans) Type() types.ResourceType { - return types.ResourceStorage -} - -// Capacity . -func (rp ResourcePlans) Capacity() map[string]int { - return rp.capacity -} - -// ApplyChangesOnNode . -func (rp ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { - node.StorageCap -= int64(len(indices)) * rp.request -} - -// RollbackChangesOnNode . -func (rp ResourcePlans) RollbackChangesOnNode(node *types.Node, indices ...int) { - node.StorageCap += int64(len(indices)) * rp.request -} - -// Dispense . -func (rp ResourcePlans) Dispense(opts resourcetypes.DispenseOptions, r *types.ResourceMeta) (*types.ResourceMeta, error) { - if rp.capacity[opts.Node.Name] <= opts.Index { - return nil, errors.WithStack(types.ErrInsufficientCap) - } - r.StorageLimit = rp.limit - r.StorageRequest = rp.request - return r, nil -} diff --git a/resources/storage/storage_test.go b/resources/storage/storage_test.go deleted file mode 100644 index c08c4ce1b..000000000 --- a/resources/storage/storage_test.go +++ /dev/null @@ -1,136 +0,0 @@ -package storage - -import ( - "context" - "testing" - - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - schedulerMocks "github.com/projecteru2/core/scheduler/mocks" - "github.com/projecteru2/core/types" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" -) - -func TestMakeRequest(t *testing.T) { - _, err := MakeRequest(types.ResourceOptions{ - StorageRequest: -1, - StorageLimit: -1, - }) - assert.NotNil(t, err) - - _, err = MakeRequest(types.ResourceOptions{ - StorageRequest: 1024, - StorageLimit: 1024, - }) - assert.Nil(t, err) - - _, err = MakeRequest(types.ResourceOptions{ - StorageRequest: 0, - StorageLimit: 1024, - }) - assert.Nil(t, err) - - _, err = MakeRequest(types.ResourceOptions{ - StorageRequest: 1024, - StorageLimit: 0, - }) - assert.Nil(t, err) - - _, err = MakeRequest(types.ResourceOptions{ - StorageRequest: 2024, - StorageLimit: 1024, - }) - assert.Nil(t, err) -} - -func TestRate(t *testing.T) { - req, err := MakeRequest(types.ResourceOptions{ - StorageRequest: 1024, - StorageLimit: 1024, - }) - assert.Nil(t, err) - node := types.Node{ - NodeMeta: types.NodeMeta{ - InitStorageCap: 1024, - }, - } - assert.Equal(t, req.Rate(node), 1.0) -} - -func TestStorage(t *testing.T) { - mockScheduler := &schedulerMocks.Scheduler{} - var ( - scheduleInfos []resourcetypes.ScheduleInfo = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "TestNode", - CPU: map[string]int64{"0": 10000, "1": 10000}, - NUMA: map[string]string{"0": "0", "1": "1"}, - NUMAMemory: map[string]int64{"0": 1024, "1": 1204}, - MemCap: 10240, - StorageCap: 10240, - }, - CPUPlan: []types.CPUMap{{"0": 10000, "1": 10000}}, - Capacity: 1, - }, - } - ) - mockScheduler.On( - "SelectStorageNodes", mock.Anything, mock.Anything, mock.Anything, - ).Return(scheduleInfos, 1, nil) - - resourceRequest, err := MakeRequest(types.ResourceOptions{ - StorageRequest: 1024, - StorageLimit: 1024, - }) - assert.NoError(t, err) - _, _, err = resourceRequest.MakeScheduler()(context.TODO(), []resourcetypes.ScheduleInfo{}) - assert.Error(t, err) - - assert.True(t, resourceRequest.Type()&types.ResourceStorage > 0) - prevSche, _ := scheduler.GetSchedulerV1() - scheduler.InitSchedulerV1(mockScheduler) - defer func() { - scheduler.InitSchedulerV1(prevSche) - }() - - sche := resourceRequest.MakeScheduler() - plans, _, err := sche(context.TODO(), scheduleInfos) - assert.Nil(t, err) - - const storage = int64(10240) - var node = types.Node{ - NodeMeta: types.NodeMeta{ - Name: "TestNode", - CPU: map[string]int64{"0": 10000, "1": 10000}, - NUMA: map[string]string{"0": "0", "1": "1"}, - NUMAMemory: map[string]int64{"0": 1024, "1": 1204}, - MemCap: 10240, - StorageCap: storage, - }, - } - - assert.True(t, plans.Type()&types.ResourceStorage > 0) - - assert.NotNil(t, plans.Capacity()) - - plans.ApplyChangesOnNode(&node, 0) - assert.Less(t, node.StorageCap, storage) - - plans.RollbackChangesOnNode(&node, 0) - assert.Equal(t, node.StorageCap, storage) - - opts := resourcetypes.DispenseOptions{ - Node: &node, - Index: 0, - } - r := &types.ResourceMeta{} - _, err = plans.Dispense(opts, r) - assert.Nil(t, err) - - opts.Node.Name = "TestNode1" - _, err = plans.Dispense(opts, r) - assert.EqualError(t, err, "cannot alloc a each node plan, not enough capacity") -} diff --git a/resources/types.go b/resources/types.go new file mode 100644 index 000000000..4c4605e65 --- /dev/null +++ b/resources/types.go @@ -0,0 +1,190 @@ +package resources + +import "github.com/projecteru2/core/types" + +// NodeCapacityInfo . +type NodeCapacityInfo struct { + NodeName string + Capacity int + + // Usage current resource usage + Usage float64 + // Rate proportion of requested resources to total + Rate float64 + + // Weight used for weighted average + Weight float64 +} + +type NodeResourceInfo struct { + Capacity types.NodeResourceArgs `json:"capacity"` + Usage types.NodeResourceArgs `json:"usage"` +} + +// GetNodesDeployCapacityRequest . +type GetNodesDeployCapacityRequest struct { + NodeNames []string `json:"node"` + ResourceOpts types.WorkloadResourceOpts `json:"resource-opts"` +} + +// GetNodesDeployCapacityResponse . +type GetNodesDeployCapacityResponse struct { + Nodes map[string]*NodeCapacityInfo `json:"nodes"` + Total int `json:"total"` +} + +// GetNodeResourceInfoRequest . +type GetNodeResourceInfoRequest struct { + NodeName string `json:"node"` + WorkloadMap map[string]types.WorkloadResourceArgs `json:"workload-map"` + Fix bool `json:"fix"` +} + +// GetNodeResourceInfoResponse , +type GetNodeResourceInfoResponse struct { + ResourceInfo *NodeResourceInfo `json:"resource_info"` + Diffs []string `json:"diffs"` +} + +// SetNodeResourceInfoRequest . +type SetNodeResourceInfoRequest struct { + NodeName string `json:"node"` + Capacity types.NodeResourceArgs `json:"capacity"` + Usage types.NodeResourceArgs `json:"usage"` +} + +// SetNodeResourceInfoResponse . +type SetNodeResourceInfoResponse struct{} + +// GetDeployArgsRequest . +type GetDeployArgsRequest struct { + NodeName string `json:"node"` + DeployCount int `json:"deploy"` + ResourceOpts types.WorkloadResourceOpts `json:"resource-opts"` +} + +// GetDeployArgsResponse . +type GetDeployArgsResponse struct { + EngineArgs []types.EngineArgs `json:"engine_args"` + ResourceArgs []types.WorkloadResourceArgs `json:"resource_args"` +} + +// GetReallocArgsRequest . +type GetReallocArgsRequest struct { + NodeName string `json:"node"` + Old types.WorkloadResourceArgs `json:"old"` + ResourceOpts types.WorkloadResourceOpts `json:"resource-opts"` +} + +// GetReallocArgsResponse . +type GetReallocArgsResponse struct { + EngineArgs types.EngineArgs `json:"engine_args"` + Delta types.WorkloadResourceArgs `json:"delta"` + ResourceArgs types.WorkloadResourceArgs `json:"resource_args"` +} + +// GetRemapArgsRequest . +type GetRemapArgsRequest struct { + NodeName string `json:"node"` + WorkloadMap map[string]types.WorkloadResourceArgs `json:"workload-map"` +} + +// GetRemapArgsResponse . +type GetRemapArgsResponse struct { + EngineArgsMap map[string]types.EngineArgs `json:"engine_args_map"` +} + +// SetNodeResourceUsageRequest . +type SetNodeResourceUsageRequest struct { + NodeName string `json:"node"` + WorkloadResourceArgs []types.WorkloadResourceArgs `json:"workload-resource-args"` + NodeResourceOpts types.NodeResourceOpts `json:"node-resource-opts"` + NodeResourceArgs types.NodeResourceArgs `json:"node-resource-args"` + Delta bool `json:"delta"` + Decr bool `json:"decr"` +} + +// SetNodeResourceUsageResponse . +type SetNodeResourceUsageResponse struct { + Before types.NodeResourceArgs `json:"before"` + After types.NodeResourceArgs `json:"after"` +} + +// SetNodeResourceCapacityRequest . +type SetNodeResourceCapacityRequest struct { + NodeName string `json:"node"` + NodeResourceOpts types.NodeResourceOpts `json:"node-resource-opts"` + NodeResourceArgs types.NodeResourceArgs `json:"node-resource-args"` + Delta bool `json:"delta"` + Decr bool `json:"decr"` +} + +// SetNodeResourceCapacityResponse . +type SetNodeResourceCapacityResponse struct { + Before types.NodeResourceArgs `json:"before"` + After types.NodeResourceArgs `json:"after"` +} + +// AddNodeRequest . +type AddNodeRequest struct { + NodeName string `json:"node"` + ResourceOpts types.NodeResourceOpts `json:"resource-opts"` +} + +// AddNodeResponse . +type AddNodeResponse struct { + Capacity types.NodeResourceArgs `json:"capacity"` + Usage types.NodeResourceArgs `json:"usage"` +} + +// RemoveNodeRequest . +type RemoveNodeRequest struct { + NodeName string `json:"node"` +} + +// RemoveNodeResponse . +type RemoveNodeResponse struct{} + +// GetMostIdleNodeRequest . +type GetMostIdleNodeRequest struct { + NodeNames []string `json:"node"` +} + +// GetMostIdleNodeResponse . +type GetMostIdleNodeResponse struct { + NodeName string `json:"node"` + Priority int `json:"priority"` +} + +// GetMetricsDescriptionRequest . +type GetMetricsDescriptionRequest struct{} + +// MetricsDescription . +type MetricsDescription struct { + Name string `json:"name"` + Help string `json:"help"` + Type string `json:"type"` + Labels []string `json:"labels"` +} + +// GetMetricsDescriptionResponse . +type GetMetricsDescriptionResponse []*MetricsDescription + +// ResolveNodeResourceInfoToMetricsRequest . +type ResolveNodeResourceInfoToMetricsRequest struct { + PodName string `json:"pod"` + NodeName string `json:"node"` + Capacity types.NodeResourceArgs `json:"capacity"` + Usage types.NodeResourceArgs `json:"usage"` +} + +// Metrics . +type Metrics struct { + Name string `json:"name"` + Labels []string `json:"labels"` + Key string `json:"key"` + Value string `json:"value"` +} + +// ResolveNodeResourceInfoToMetricsResponse . +type ResolveNodeResourceInfoToMetricsResponse []*Metrics diff --git a/resources/types/mocks/ResourcePlans.go b/resources/types/mocks/ResourcePlans.go deleted file mode 100644 index bf5633171..000000000 --- a/resources/types/mocks/ResourcePlans.go +++ /dev/null @@ -1,89 +0,0 @@ -// Code generated by mockery v1.0.0. DO NOT EDIT. - -package mocks - -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 { - mock.Mock -} - -// ApplyChangesOnNode provides a mock function with given fields: _a0, _a1 -func (_m *ResourcePlans) ApplyChangesOnNode(_a0 *types.Node, _a1 ...int) { - _va := make([]interface{}, len(_a1)) - for _i := range _a1 { - _va[_i] = _a1[_i] - } - var _ca []interface{} - _ca = append(_ca, _a0) - _ca = append(_ca, _va...) - _m.Called(_ca...) -} - -// Capacity provides a mock function with given fields: -func (_m *ResourcePlans) Capacity() map[string]int { - ret := _m.Called() - - var r0 map[string]int - if rf, ok := ret.Get(0).(func() map[string]int); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(map[string]int) - } - } - - return r0 -} - -// Dispense provides a mock function with given fields: _a0, _a1 -func (_m *ResourcePlans) Dispense(_a0 resourcestypes.DispenseOptions, _a1 *types.ResourceMeta) (*types.ResourceMeta, error) { - ret := _m.Called(_a0, _a1) - - var r0 *types.ResourceMeta - if rf, ok := ret.Get(0).(func(resourcestypes.DispenseOptions, *types.ResourceMeta) *types.ResourceMeta); ok { - r0 = rf(_a0, _a1) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*types.ResourceMeta) - } - } - - 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 -func (_m *ResourcePlans) RollbackChangesOnNode(_a0 *types.Node, _a1 ...int) { - _va := make([]interface{}, len(_a1)) - for _i := range _a1 { - _va[_i] = _a1[_i] - } - var _ca []interface{} - _ca = append(_ca, _a0) - _ca = append(_ca, _va...) - _m.Called(_ca...) -} - -// Type provides a mock function with given fields: -func (_m *ResourcePlans) Type() types.ResourceType { - ret := _m.Called() - - var r0 types.ResourceType - if rf, ok := ret.Get(0).(func() types.ResourceType); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(types.ResourceType) - } - - return r0 -} diff --git a/resources/types/types.go b/resources/types/types.go deleted file mode 100644 index 1291b6029..000000000 --- a/resources/types/types.go +++ /dev/null @@ -1,85 +0,0 @@ -package types - -import ( - "context" - - "github.com/projecteru2/core/types" -) - -const supported = 3 - -// ResourceRequests . -type ResourceRequests [supported]ResourceRequest - -// MainResourceType calculates main resource type -func (rr ResourceRequests) MainResourceType() (mainType types.ResourceType) { - for _, req := range rr { - if req.Type()&types.ResourceCPUBind != 0 { - mainType |= types.ResourceCPU - } - if req.Type()&types.ResourceScheduledVolume != 0 { - mainType |= types.ResourceVolume - } - } - if mainType == 0 { - mainType = types.ResourceMemory - } - return mainType -} - -// MainRateOnNode calculates resource consumption rate for this request on a node -func (rr ResourceRequests) MainRateOnNode(node types.Node) (rate float64) { - mainType := rr.MainResourceType() - for _, req := range rr { - if req.Type()&mainType != 0 { - rate += req.Rate(node) - } - } - return -} - -// MainUsageOnNode calculates current resource usage rate based on MainType -func (rr ResourceRequests) MainUsageOnNode(node types.Node) (usage float64) { - mainType := rr.MainResourceType() - for t, use := range node.ResourceUsages() { - if t&mainType != 0 { - usage += use - } - } - return -} - -// ResourceRequest . -type ResourceRequest interface { - Type() types.ResourceType - Validate() error - MakeScheduler() SchedulerV2 - Rate(types.Node) float64 -} - -// SchedulerV2 . -type SchedulerV2 func(context.Context, []ScheduleInfo) (ResourcePlans, int, error) - -// DispenseOptions . -type DispenseOptions struct { - *types.Node - Index int -} - -// ResourcePlans . -type ResourcePlans interface { - Type() types.ResourceType - Capacity() map[string]int - ApplyChangesOnNode(*types.Node, ...int) - RollbackChangesOnNode(*types.Node, ...int) - Dispense(DispenseOptions, *types.ResourceMeta) (*types.ResourceMeta, error) -} - -// ScheduleInfo for scheduler -type ScheduleInfo struct { - types.NodeMeta - - CPUPlan []types.CPUMap - VolumePlans []types.VolumePlan // {{"AUTO:/data:rw:1024": "/mnt0:/data:rw:1024"}} - Capacity int // 可以部署几个 -} diff --git a/resources/types/types_test.go b/resources/types/types_test.go deleted file mode 100644 index 130307336..000000000 --- a/resources/types/types_test.go +++ /dev/null @@ -1,52 +0,0 @@ -package types - -import ( - "testing" - - "github.com/projecteru2/core/types" - - "github.com/stretchr/testify/assert" -) - -type mockResourceRequest struct { - t types.ResourceType - rate float64 -} - -func (m *mockResourceRequest) Type() types.ResourceType { return m.t } - -func (m *mockResourceRequest) Validate() error { return nil } - -func (m *mockResourceRequest) MakeScheduler() SchedulerV2 { return nil } - -func (m *mockResourceRequest) Rate(node types.Node) float64 { return m.rate } - -func TestResourceRequestsMethod(t *testing.T) { - node := types.Node{ - NodeMeta: types.NodeMeta{ - InitCPU: types.CPUMap{"0": 100}, - InitMemCap: 100, - InitStorageCap: 200, - InitVolume: types.VolumeMap{"/sda": 200}, - }, - CPUUsed: 0.5, - } - rrs := ResourceRequests{ - &mockResourceRequest{t: types.ResourceCPUBind}, - &mockResourceRequest{t: types.ResourceStorage}, - &mockResourceRequest{t: types.ResourceScheduledVolume}, - } - assert.EqualValues(t, types.ResourceCPU|types.ResourceVolume, rrs.MainResourceType()) - assert.EqualValues(t, 0, rrs.MainRateOnNode(node)) - assert.EqualValues(t, 0.5, rrs.MainUsageOnNode(node)) - - rrs = ResourceRequests{ - &mockResourceRequest{t: types.ResourceMemory}, - &mockResourceRequest{t: types.ResourceStorage}, - &mockResourceRequest{t: types.ResourceVolume}, - } - assert.EqualValues(t, types.ResourceMemory, rrs.MainResourceType()) - assert.EqualValues(t, 0, rrs.MainRateOnNode(node)) - assert.EqualValues(t, 1, rrs.MainUsageOnNode(node)) - -} diff --git a/resources/types/utils.go b/resources/types/utils.go deleted file mode 100644 index 9b1347157..000000000 --- a/resources/types/utils.go +++ /dev/null @@ -1,10 +0,0 @@ -package types - -// GetCapacity . -func GetCapacity(scheduleInfos []ScheduleInfo) map[string]int { - capacity := make(map[string]int) - for _, scheduleInfo := range scheduleInfos { - capacity[scheduleInfo.Name] = scheduleInfo.Capacity - } - return capacity -} diff --git a/resources/types/utils_test.go b/resources/types/utils_test.go deleted file mode 100644 index 9eb9fe25a..000000000 --- a/resources/types/utils_test.go +++ /dev/null @@ -1,19 +0,0 @@ -package types - -import ( - "testing" - - "github.com/projecteru2/core/types" - - "github.com/stretchr/testify/assert" -) - -func TestGetCapacity(t *testing.T) { - nodesInfo := []ScheduleInfo{ - {NodeMeta: types.NodeMeta{Name: "1"}, Capacity: 1}, - {NodeMeta: types.NodeMeta{Name: "2"}, Capacity: 1}, - } - r := GetCapacity(nodesInfo) - assert.Equal(t, r["1"], 1) - assert.Equal(t, r["2"], 1) -} diff --git a/resources/volume/models/alloc.go b/resources/volume/models/alloc.go new file mode 100644 index 000000000..420a32962 --- /dev/null +++ b/resources/volume/models/alloc.go @@ -0,0 +1,105 @@ +package models + +import ( + "context" + + "github.com/pkg/errors" + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/volume/schedule" + "github.com/projecteru2/core/resources/volume/types" +) + +// GetDeployArgs . +func (v *Volume) GetDeployArgs(ctx context.Context, node string, deployCount int, opts *types.WorkloadResourceOpts) ([]*types.EngineArgs, []*types.WorkloadResourceArgs, error) { + if err := opts.Validate(); err != nil { + logrus.Errorf("[Alloc] invalid resource opts %+v, err: %v", opts, err) + return nil, nil, err + } + + resourceInfo, err := v.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[Alloc] failed to get resource info of node %v, err: %v", node, err) + return nil, nil, err + } + + return v.doAlloc(resourceInfo, deployCount, opts) +} + +func getVolumePlanLimit(volumeRequest types.VolumeBindings, volumeLimit types.VolumeBindings, volumePlan types.VolumePlan) types.VolumePlan { + volumePlanLimit := types.VolumePlan{} + + volumeBindingToVolumeMap := map[[3]string]types.VolumeMap{} + for binding, volumeMap := range volumePlan { + volumeBindingToVolumeMap[binding.GetMapKey()] = volumeMap + } + + for index, binding := range volumeLimit { + if !binding.RequireSchedule() { + continue + } + if volumeMap, ok := volumeBindingToVolumeMap[binding.GetMapKey()]; ok { + volumePlanLimit[binding] = types.VolumeMap{volumeMap.GetDevice(): volumeMap.GetSize() + binding.SizeInBytes - volumeRequest[index].SizeInBytes} + } + } + return volumePlanLimit +} + +func (v *Volume) doAlloc(resourceInfo *types.NodeResourceInfo, deployCount int, opts *types.WorkloadResourceOpts) ([]*types.EngineArgs, []*types.WorkloadResourceArgs, error) { + // check if storage is enough + if opts.StorageRequest > 0 { + storageCapacity := int((resourceInfo.Capacity.Storage - resourceInfo.Usage.Storage) / opts.StorageRequest) + if storageCapacity < deployCount { + return nil, nil, errors.Wrapf(types.ErrInsufficientResource, "not enough storage, request: %v, available: %v", opts.StorageRequest, storageCapacity) + } + } + + resEngineArgs := []*types.EngineArgs{} + resResourceArgs := []*types.WorkloadResourceArgs{} + + // if volume is not required + if len(opts.VolumesRequest) == 0 { + for i := 0; i < deployCount; i++ { + resEngineArgs = append(resEngineArgs, &types.EngineArgs{ + Storage: opts.StorageLimit, + }) + resResourceArgs = append(resResourceArgs, &types.WorkloadResourceArgs{ + StorageRequest: opts.StorageRequest, + StorageLimit: opts.StorageLimit, + }) + } + return resEngineArgs, resResourceArgs, nil + } + + volumePlans := schedule.GetVolumePlans(resourceInfo, opts.VolumesRequest, v.Config.Scheduler.MaxDeployCount) + if len(volumePlans) < deployCount { + return nil, nil, errors.Wrapf(types.ErrInsufficientResource, "not enough volume plan, need %v, available %v", deployCount, len(volumePlans)) + } + + volumePlans = volumePlans[:deployCount] + volumeSizeLimitMap := map[*types.VolumeBinding]int64{} + for _, binding := range opts.VolumesLimit { + volumeSizeLimitMap[binding] = binding.SizeInBytes + } + + for _, volumePlan := range volumePlans { + engineArgs := &types.EngineArgs{Storage: opts.StorageLimit} + for _, binding := range opts.VolumesLimit.ApplyPlan(volumePlan) { + engineArgs.Volumes = append(engineArgs.Volumes, binding.ToString(true)) + } + + resourceArgs := &types.WorkloadResourceArgs{ + VolumesRequest: opts.VolumesRequest, + VolumesLimit: opts.VolumesLimit, + VolumePlanRequest: volumePlan, + VolumePlanLimit: getVolumePlanLimit(opts.VolumesLimit, opts.VolumesLimit, volumePlan), + StorageRequest: opts.StorageRequest, + StorageLimit: opts.StorageLimit, + } + + resEngineArgs = append(resEngineArgs, engineArgs) + resResourceArgs = append(resResourceArgs, resourceArgs) + } + + return resEngineArgs, resResourceArgs, nil +} diff --git a/resources/volume/models/alloc_test.go b/resources/volume/models/alloc_test.go new file mode 100644 index 000000000..bf047eab2 --- /dev/null +++ b/resources/volume/models/alloc_test.go @@ -0,0 +1,76 @@ +package models + +import ( + "context" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/volume/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestAlloc(t *testing.T) { + ctx := context.Background() + + volume := newTestVolume(t) + nodes := generateNodes(t, volume, 1) + node := nodes[0] + + // invalid resource opt + resourceOpts := &types.WorkloadResourceOpts{ + StorageRequest: -1, + } + _, _, err := volume.GetDeployArgs(ctx, node, 1, resourceOpts) + assert.ErrorIs(t, err, types.ErrInvalidStorage) + + // invalid node + resourceOpts = &types.WorkloadResourceOpts{ + VolumesLimit: generateVolumeBindings(t, []string{ + "/etc:/etc", + }), + } + _, _, err = volume.GetDeployArgs(ctx, "fake-node", 1, resourceOpts) + assert.ErrorIs(t, err, coretypes.ErrBadCount) + + // storage is not enough + resourceOpts = &types.WorkloadResourceOpts{ + StorageRequest: 4 * units.TiB, + VolumesLimit: generateVolumeBindings(t, []string{ + "/etc:/etc", + }), + } + _, _, err = volume.GetDeployArgs(ctx, node, 2, resourceOpts) + assert.ErrorIs(t, err, types.ErrInsufficientResource) + + // normal case + resourceOpts = &types.WorkloadResourceOpts{ + StorageRequest: 10 * units.GiB, + } + + _, _, err = volume.GetDeployArgs(ctx, node, 100, resourceOpts) + assert.Nil(t, err) + + // insufficient volume + resourceOpts = &types.WorkloadResourceOpts{ + StorageRequest: 1 * units.GiB, + VolumesLimit: generateVolumeBindings(t, []string{ + "AUTO:/dir0:rwm:1GiB", + "/etc:etc", + }), + } + _, _, err = volume.GetDeployArgs(ctx, node, 3, resourceOpts) + assert.ErrorIs(t, err, types.ErrInsufficientResource) + + // normal case + resourceOpts = &types.WorkloadResourceOpts{ + StorageRequest: 1 * units.GiB, + VolumesLimit: generateVolumeBindings(t, []string{ + "AUTO:/dir0:rwm:1GiB", + "AUTO:/dir1:rw:1GiB", + }), + } + _, _, err = volume.GetDeployArgs(ctx, node, 2, resourceOpts) + assert.Nil(t, err) +} diff --git a/resources/volume/models/capacity.go b/resources/volume/models/capacity.go new file mode 100644 index 000000000..6ff66c390 --- /dev/null +++ b/resources/volume/models/capacity.go @@ -0,0 +1,74 @@ +package models + +import ( + "context" + "math" + + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/volume/schedule" + "github.com/projecteru2/core/resources/volume/types" +) + +// GetNodesDeployCapacity . +func (v *Volume) GetNodesDeployCapacity(ctx context.Context, nodes []string, opts *types.WorkloadResourceOpts) (map[string]*types.NodeCapacityInfo, int, error) { + if err := opts.Validate(); err != nil { + logrus.Errorf("[GetNodesDeployCapacity] invalid resource opts %+v, err: %v", opts, err) + return nil, 0, err + } + + capacityInfoMap := map[string]*types.NodeCapacityInfo{} + total := 0 + for _, node := range nodes { + resourceInfo, err := v.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[GetNodesDeployCapacity] failed to get resource info of node %v, err: %v", node, err) + return nil, 0, err + } + capacityInfo := v.doGetNodeCapacityInfo(ctx, node, resourceInfo, opts) + if capacityInfo.Capacity > 0 { + capacityInfoMap[node] = capacityInfo + if total == math.MaxInt64 || capacityInfo.Capacity == math.MaxInt64 { + total = math.MaxInt64 + } else { + total += capacityInfo.Capacity + } + } + } + + return capacityInfoMap, total, nil +} + +func (v *Volume) doGetNodeCapacityInfo(ctx context.Context, node string, resourceInfo *types.NodeResourceInfo, opts *types.WorkloadResourceOpts) *types.NodeCapacityInfo { + capacityInfo := &types.NodeCapacityInfo{ + Node: node, + Weight: 1, + } + + // get volume capacity + volumePlans := schedule.GetVolumePlans(resourceInfo, opts.VolumesRequest, v.Config.Scheduler.MaxDeployCount) + capacityInfo.Capacity = len(volumePlans) + + // get storage capacity + if opts.StorageRequest > 0 { + storageCapacity := int((resourceInfo.Capacity.Storage - resourceInfo.Usage.Storage) / opts.StorageRequest) + if storageCapacity < capacityInfo.Capacity { + capacityInfo.Capacity = storageCapacity + } + } + + // get usage and rate + if resourceInfo.Capacity.Volumes.Total() == 0 && resourceInfo.Capacity.Storage == 0 { + return capacityInfo + } + + if len(opts.VolumesRequest) > 0 || opts.StorageRequest == 0 { + capacityInfo.Usage = float64(resourceInfo.Usage.Volumes.Total()) / float64(resourceInfo.Capacity.Volumes.Total()) + capacityInfo.Rate = float64(opts.VolumesRequest.TotalSize()) / float64(resourceInfo.Capacity.Volumes.Total()) + } else if opts.StorageRequest > 0 { + capacityInfo.Usage = float64(resourceInfo.Usage.Storage) / float64(resourceInfo.Capacity.Storage) + capacityInfo.Rate = float64(opts.StorageRequest) / float64(resourceInfo.Capacity.Storage) + } + + return capacityInfo +} diff --git a/resources/volume/models/capacity_test.go b/resources/volume/models/capacity_test.go new file mode 100644 index 000000000..c4e2b2415 --- /dev/null +++ b/resources/volume/models/capacity_test.go @@ -0,0 +1,64 @@ +package models + +import ( + "context" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/volume/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestGetNodesDeployCapacity(t *testing.T) { + ctx := context.Background() + + volume := newTestVolume(t) + nodes := generateNodes(t, volume, 10) + + // invalid opts + resourceOpts := &types.WorkloadResourceOpts{ + StorageRequest: -1, + } + + _, _, err := volume.GetNodesDeployCapacity(ctx, nodes, resourceOpts) + assert.ErrorIs(t, err, types.ErrInvalidStorage) + + // invalid node + resourceOpts = &types.WorkloadResourceOpts{ + StorageRequest: 1, + } + _, _, err = volume.GetNodesDeployCapacity(ctx, []string{"invalid"}, resourceOpts) + assert.ErrorIs(t, err, coretypes.ErrBadCount) + + // no volume request + resourceOpts = &types.WorkloadResourceOpts{ + StorageLimit: units.TiB, + } + _, total, err := volume.GetNodesDeployCapacity(ctx, nodes, resourceOpts) + assert.NoError(t, err) + assert.Equal(t, total, 30) + + // no storage request + resourceOpts = &types.WorkloadResourceOpts{ + VolumesLimit: generateVolumeBindings(t, []string{ + "AUTO:/dir0:rwm:1G", + }), + } + _, total, err = volume.GetNodesDeployCapacity(ctx, nodes, resourceOpts) + assert.NoError(t, err) + assert.Equal(t, total, 20) + + // mixed + resourceOpts = &types.WorkloadResourceOpts{ + VolumesLimit: generateVolumeBindings(t, []string{ + "AUTO:/dir0:rwm:1G", + "AUTO:/dir1:rwm:1G", + }), + StorageRequest: 1 * units.TiB, + } + _, total, err = volume.GetNodesDeployCapacity(ctx, nodes, resourceOpts) + assert.NoError(t, err) + assert.Equal(t, total, 20) +} diff --git a/resources/volume/models/idle.go b/resources/volume/models/idle.go new file mode 100644 index 000000000..9ee8ed366 --- /dev/null +++ b/resources/volume/models/idle.go @@ -0,0 +1,12 @@ +package models + +import ( + "context" +) + +const priority = 1 + +// GetMostIdleNode . +func (v *Volume) GetMostIdleNode(ctx context.Context, nodes []string) (string, int, error) { + return "", priority, nil +} diff --git a/resources/volume/models/info.go b/resources/volume/models/info.go new file mode 100644 index 000000000..698126205 --- /dev/null +++ b/resources/volume/models/info.go @@ -0,0 +1,195 @@ +package models + +import ( + "context" + "encoding/json" + "fmt" + + "github.com/sanity-io/litter" + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/volume/types" +) + +const NodeResourceInfoKey = "/resource/volume/%s" + +// GetNodeResourceInfo . +func (v *Volume) GetNodeResourceInfo(ctx context.Context, node string, workloadResourceMap *types.WorkloadResourceArgsMap, fix bool) (*types.NodeResourceInfo, []string, error) { + resourceInfo, err := v.doGetNodeResourceInfo(ctx, node) + if err != nil { + return nil, nil, err + } + + diffs := []string{} + + totalVolumeMap := types.VolumeMap{} + totalStorageUsage := int64(0) + + if workloadResourceMap != nil { + for _, args := range *workloadResourceMap { + for _, volumeMap := range args.VolumePlanRequest { + totalVolumeMap.Add(volumeMap) + } + totalStorageUsage += args.StorageRequest + } + } + + if resourceInfo.Usage.Storage != totalStorageUsage { + diffs = append(diffs, fmt.Sprintf("node.Storage != sum(workload.Storage): %v != %v", resourceInfo.Usage.Storage, totalStorageUsage)) + } + + for volume, size := range resourceInfo.Usage.Volumes { + if totalVolumeMap[volume] != size { + diffs = append(diffs, fmt.Sprintf("node.Volumes[%s] != sum(workload.Volumes[%s]): %v != %v", volume, volume, size, totalVolumeMap[volume])) + } + } + for volume, size := range totalVolumeMap { + if vol, ok := resourceInfo.Usage.Volumes[volume]; !ok && vol != size { + diffs = append(diffs, fmt.Sprintf("node.Volumes[%s] != sum(workload.Volumes[%s]): %v != %v", volume, volume, resourceInfo.Usage.Volumes[volume], size)) + } + } + + if fix { + resourceInfo.Usage = &types.NodeResourceArgs{ + Volumes: totalVolumeMap, + Storage: totalStorageUsage, + } + if err = v.doSetNodeResourceInfo(ctx, node, resourceInfo); err != nil { + logrus.Warnf("[GetNodeResourceInfo] failed to fix node resource, err: %v", err) + diffs = append(diffs, "fix failed") + } + } + + return resourceInfo, diffs, nil +} + +// priority: node resource opts > node resource args > workload resource args list +func (v *Volume) calculateNodeResourceArgs(origin *types.NodeResourceArgs, nodeResourceOpts *types.NodeResourceOpts, nodeResourceArgs *types.NodeResourceArgs, workloadResourceArgs []*types.WorkloadResourceArgs, delta bool, incr bool) (res *types.NodeResourceArgs) { + if origin == nil || !delta { + res = (&types.NodeResourceArgs{}).DeepCopy() + } else { + res = origin.DeepCopy() + } + + if nodeResourceOpts != nil { + nodeResourceArgs := &types.NodeResourceArgs{ + Volumes: nodeResourceOpts.Volumes, + Storage: nodeResourceOpts.Storage, + } + + if incr { + res.Add(nodeResourceArgs) + } else { + res.Sub(nodeResourceArgs) + } + return res + } + + if nodeResourceArgs != nil { + if incr { + res.Add(nodeResourceArgs) + } else { + res.Sub(nodeResourceArgs) + } + return res + } + + for _, args := range workloadResourceArgs { + nodeResourceArgs := &types.NodeResourceArgs{ + Volumes: map[string]int64{}, + Storage: args.StorageRequest, + } + for _, volumeMap := range args.VolumePlanRequest { + nodeResourceArgs.Volumes.Add(volumeMap) + } + if incr { + res.Add(nodeResourceArgs) + } else { + res.Sub(nodeResourceArgs) + } + } + return res +} + +// SetNodeResourceUsage . +func (v *Volume) SetNodeResourceUsage(ctx context.Context, node string, nodeResourceOpts *types.NodeResourceOpts, nodeResourceArgs *types.NodeResourceArgs, workloadResourceArgs []*types.WorkloadResourceArgs, delta bool, incr bool) (before *types.NodeResourceArgs, after *types.NodeResourceArgs, err error) { + resourceInfo, err := v.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[SetNodeResourceInfo] failed to get resource info of node %v, err: %v", node, err) + return nil, nil, err + } + + before = resourceInfo.Usage.DeepCopy() + resourceInfo.Usage = v.calculateNodeResourceArgs(resourceInfo.Usage, nodeResourceOpts, nodeResourceArgs, workloadResourceArgs, delta, incr) + + if err := v.doSetNodeResourceInfo(ctx, node, resourceInfo); err != nil { + return nil, nil, err + } + return before, resourceInfo.Usage, nil +} + +// SetNodeResourceCapacity . +func (v *Volume) SetNodeResourceCapacity(ctx context.Context, node string, nodeResourceOpts *types.NodeResourceOpts, nodeResourceArgs *types.NodeResourceArgs, delta bool, incr bool) (before *types.NodeResourceArgs, after *types.NodeResourceArgs, err error) { + resourceInfo, err := v.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[SetNodeResourceInfo] failed to get resource info of node %v, err: %v", node, err) + return nil, nil, err + } + + before = resourceInfo.Capacity.DeepCopy() + if nodeResourceOpts != nil && !delta { + nodeResourceOpts.SkipEmpty(resourceInfo.Capacity) + } + resourceInfo.Capacity = v.calculateNodeResourceArgs(resourceInfo.Capacity, nodeResourceOpts, nodeResourceArgs, nil, delta, incr) + if delta { + resourceInfo.Capacity.RemoveEmpty() + } + + if err := v.doSetNodeResourceInfo(ctx, node, resourceInfo); err != nil { + return nil, nil, err + } + return before, resourceInfo.Capacity, nil +} + +// SetNodeResourceInfo . +func (v *Volume) SetNodeResourceInfo(ctx context.Context, node string, resourceCapacity *types.NodeResourceArgs, resourceUsage *types.NodeResourceArgs) error { + resourceInfo := &types.NodeResourceInfo{ + Capacity: resourceCapacity, + Usage: resourceUsage, + } + + return v.doSetNodeResourceInfo(ctx, node, resourceInfo) +} + +func (v *Volume) doGetNodeResourceInfo(ctx context.Context, node string) (*types.NodeResourceInfo, error) { + resourceInfo := &types.NodeResourceInfo{} + resp, err := v.store.GetOne(ctx, fmt.Sprintf(NodeResourceInfoKey, node)) + if err != nil { + logrus.Errorf("[doGetNodeResourceInfo] failed to get node resource info of node %v, err: %v", node, err) + return nil, err + } + if err = json.Unmarshal(resp.Value, resourceInfo); err != nil { + logrus.Errorf("[doGetNodeResourceInfo] failed to unmarshal node resource info of node %v, err: %v", node, err) + return nil, err + } + return resourceInfo, nil +} + +func (v *Volume) doSetNodeResourceInfo(ctx context.Context, node string, resourceInfo *types.NodeResourceInfo) error { + if err := resourceInfo.Validate(); err != nil { + logrus.Errorf("[doSetNodeResourceInfo] invalid resource info %v, err: %v", litter.Sdump(resourceInfo), err) + return err + } + + data, err := json.Marshal(resourceInfo) + if err != nil { + logrus.Errorf("[doSetNodeResourceInfo] faield to marshal resource info %+v, err: %v", resourceInfo, err) + return err + } + + if _, err = v.store.Put(ctx, fmt.Sprintf(NodeResourceInfoKey, node), string(data)); err != nil { + logrus.Errorf("[doSetNodeResourceInfo] faield to put resource info %+v, err: %v", resourceInfo, err) + return err + } + return nil +} diff --git a/resources/volume/models/info_test.go b/resources/volume/models/info_test.go new file mode 100644 index 000000000..04b4cfa2e --- /dev/null +++ b/resources/volume/models/info_test.go @@ -0,0 +1,210 @@ +package models + +import ( + "context" + "testing" + + "github.com/docker/go-units" + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/volume/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestGetNodeResourceInfo(t *testing.T) { + ctx := context.Background() + + volume := newTestVolume(t) + nodes := generateNodes(t, volume, 10) + node := nodes[0] + + // invalid node + _, _, err := volume.GetNodeResourceInfo(ctx, "xxx", nil, false) + assert.True(t, errors.Is(err, coretypes.ErrBadCount)) + + // normal case + resourceInfo, diffs, err := volume.GetNodeResourceInfo(ctx, node, nil, false) + assert.Nil(t, err) + assert.Equal(t, len(diffs), 3) + + bindings := generateVolumeBindings(t, []string{ + "AUTO:/dir0:rw:1GiB", + }) + + resourceInfo, diffs, err = volume.GetNodeResourceInfo(ctx, node, &types.WorkloadResourceArgsMap{ + "x-workload": { + VolumesRequest: bindings, + VolumesLimit: bindings, + VolumePlanRequest: types.VolumePlan{ + bindings[0]: types.VolumeMap{"/data0": units.GiB}, + }, + VolumePlanLimit: types.VolumePlan{ + bindings[0]: types.VolumeMap{"/data0": units.GiB}, + }, + StorageRequest: units.GiB, + StorageLimit: units.GiB, + }, + }, true) + assert.Nil(t, err) + assert.Equal(t, len(diffs), 3) + assert.Equal(t, resourceInfo.Usage, &types.NodeResourceArgs{ + Volumes: types.VolumeMap{ + "/data0": units.GiB, + }, + Storage: units.GiB, + }) +} + +func TestSetNodeResourceInfo(t *testing.T) { + ctx := context.Background() + + volume := newTestVolume(t) + nodes := generateNodes(t, volume, 10) + node := nodes[0] + + resourceInfo, _, err := volume.GetNodeResourceInfo(ctx, node, nil, false) + assert.Nil(t, err) + + err = volume.SetNodeResourceInfo(ctx, "node-x", resourceInfo.Capacity, resourceInfo.Usage) + assert.Nil(t, err) +} + +func TestSetNodeResourceUsage(t *testing.T) { + ctx := context.Background() + + volume := newTestVolume(t) + nodes := generateNodes(t, volume, 10) + node := nodes[0] + + _, _, err := volume.GetNodeResourceInfo(ctx, node, nil, false) + assert.Nil(t, err) + + var after *types.NodeResourceArgs + + nodeResourceOpts := &types.NodeResourceOpts{ + Volumes: types.VolumeMap{"/data0": units.GiB}, + Storage: units.GiB, + } + + nodeResourceArgs := &types.NodeResourceArgs{ + Volumes: types.VolumeMap{"/data0": units.GiB}, + Storage: units.GiB, + } + + workloadResourceArgs := []*types.WorkloadResourceArgs{ + { + VolumePlanRequest: types.VolumePlan{ + generateVolumeBindings(t, []string{"AUTO:/dir0:rw:1GiB"})[0]: types.VolumeMap{"/data0": units.GiB}, + }, + StorageRequest: units.GiB, + }, + } + + originResourceUsage := &types.NodeResourceArgs{ + Volumes: types.VolumeMap{"/data0": 200 * units.GiB, "/data1": 300 * units.GiB}, + Storage: 500 * units.GiB, + } + + afterSetNodeResourceUsageDelta := &types.NodeResourceArgs{ + Volumes: types.VolumeMap{ + "/data0": 201 * units.GiB, + "/data1": 300 * units.GiB, + }, + Storage: 501 * units.GiB, + } + + _, after, err = volume.SetNodeResourceUsage(ctx, node, nodeResourceOpts, nil, nil, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDelta) + + _, after, err = volume.SetNodeResourceUsage(ctx, node, nodeResourceOpts, nil, nil, true, false) + assert.Nil(t, err) + assert.Equal(t, originResourceUsage, after) + + _, after, err = volume.SetNodeResourceUsage(ctx, node, nil, nodeResourceArgs, nil, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDelta) + + _, after, err = volume.SetNodeResourceUsage(ctx, node, nil, nodeResourceArgs, nil, true, false) + assert.Nil(t, err) + assert.Equal(t, originResourceUsage, after) + + _, after, err = volume.SetNodeResourceUsage(ctx, node, nil, nil, workloadResourceArgs, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDelta) + + _, after, err = volume.SetNodeResourceUsage(ctx, node, nil, nil, workloadResourceArgs, true, false) + assert.Nil(t, err) + assert.Equal(t, originResourceUsage, after) + + _, after, err = volume.SetNodeResourceUsage(ctx, node, nil, nil, nil, true, false) + assert.Nil(t, err) + assert.Equal(t, originResourceUsage, after) + + _, after, err = volume.SetNodeResourceUsage(ctx, node, nil, nodeResourceArgs, nil, false, true) + assert.Nil(t, err) + assert.Equal(t, nodeResourceArgs.DeepCopy(), after.DeepCopy()) +} + +func TestNodeResourceCapacity(t *testing.T) { + ctx := context.Background() + + volume := newTestVolume(t) + nodes := generateNodes(t, volume, 10) + node := nodes[0] + + _, _, err := volume.GetNodeResourceInfo(ctx, node, nil, false) + assert.Nil(t, err) + + var after *types.NodeResourceArgs + + nodeResourceOptsDelta := &types.NodeResourceOpts{ + Volumes: types.VolumeMap{"/data4": units.TiB}, + Storage: units.TiB, + } + + nodeResourceArgsDelta := &types.NodeResourceArgs{ + Volumes: types.VolumeMap{"/data4": units.TiB}, + Storage: units.TiB, + } + + originResourceCapacity := &types.NodeResourceArgs{ + Volumes: types.VolumeMap{"/data0": units.TiB, "/data1": units.TiB, "/data2": units.TiB, "/data3": units.TiB}, + Storage: 4 * units.TiB, + } + + originResourceCapacityArgs := &types.NodeResourceArgs{ + Volumes: types.VolumeMap{"/data0": units.TiB, "/data1": units.TiB, "/data2": units.TiB, "/data3": units.TiB}, + Storage: 4 * units.TiB, + } + + afterSetNodeResourceUsageDelta := &types.NodeResourceArgs{ + Volumes: types.VolumeMap{"/data0": units.TiB, "/data1": units.TiB, "/data2": units.TiB, "/data3": units.TiB, "/data4": units.TiB}, + Storage: 5 * units.TiB, + } + + _, after, err = volume.SetNodeResourceCapacity(ctx, node, nodeResourceOptsDelta, nil, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDelta) + + _, after, err = volume.SetNodeResourceCapacity(ctx, node, nodeResourceOptsDelta, nil, true, false) + assert.Nil(t, err) + assert.Equal(t, after, originResourceCapacity) + + _, after, err = volume.SetNodeResourceCapacity(ctx, node, nil, nodeResourceArgsDelta, true, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDelta) + + _, after, err = volume.SetNodeResourceCapacity(ctx, node, nil, nodeResourceArgsDelta, true, false) + assert.Nil(t, err) + assert.Equal(t, after, originResourceCapacity) + + _, after, err = volume.SetNodeResourceCapacity(ctx, node, nil, afterSetNodeResourceUsageDelta, false, true) + assert.Nil(t, err) + assert.Equal(t, after, afterSetNodeResourceUsageDelta) + + _, after, err = volume.SetNodeResourceCapacity(ctx, node, nil, originResourceCapacityArgs, false, true) + assert.Nil(t, err) + assert.Equal(t, after, originResourceCapacity) +} diff --git a/resources/volume/models/metrics.go b/resources/volume/models/metrics.go new file mode 100644 index 000000000..2db4b6e19 --- /dev/null +++ b/resources/volume/models/metrics.go @@ -0,0 +1,46 @@ +package models + +import ( + "fmt" + "strings" + + "github.com/projecteru2/core/resources/volume/types" +) + +// GetMetricsDescription . +func (v *Volume) GetMetricsDescription() []map[string]interface{} { + return []map[string]interface{}{ + { + "name": "storage_used", + "help": "node used storage.", + "type": "gauge", + "labels": []string{"podname", "nodename"}, + }, + { + "name": "storage_capacity", + "help": "node available storage.", + "type": "gauge", + "labels": []string{"podname", "nodename"}, + }, + } +} + +func (v *Volume) ResolveNodeResourceInfoToMetrics(podName string, nodeName string, nodeResourceCapacity *types.NodeResourceArgs, nodeResourceUsage *types.NodeResourceArgs) []map[string]interface{} { + cleanedNodeName := strings.ReplaceAll(nodeName, ".", "_") + metrics := []map[string]interface{}{ + { + "name": "storage_used", + "labels": []string{podName, nodeName}, + "value": fmt.Sprintf("%v", nodeResourceUsage.Storage), + "key": fmt.Sprintf("core.node.%s.storage.used", cleanedNodeName), + }, + { + "name": "storage_capacity", + "labels": []string{podName, nodeName}, + "value": fmt.Sprintf("%v", nodeResourceCapacity.Storage), + "key": fmt.Sprintf("core.node.%s.storage.used", cleanedNodeName), + }, + } + + return metrics +} diff --git a/resources/volume/models/node.go b/resources/volume/models/node.go new file mode 100644 index 000000000..b3c2671f4 --- /dev/null +++ b/resources/volume/models/node.go @@ -0,0 +1,43 @@ +package models + +import ( + "context" + "fmt" + + "github.com/pkg/errors" + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/volume/types" + coretypes "github.com/projecteru2/core/types" +) + +// AddNode . +func (v *Volume) AddNode(ctx context.Context, node string, resourceOpts *types.NodeResourceOpts) (*types.NodeResourceInfo, error) { + if _, err := v.doGetNodeResourceInfo(ctx, node); err != nil { + if !errors.Is(err, coretypes.ErrBadCount) { + logrus.Errorf("[AddNode] failed to get resource info of node %v, err: %v", node, err) + return nil, err + } + } else { + return nil, types.ErrNodeExists + } + + resourceInfo := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + Volumes: resourceOpts.Volumes, + Storage: resourceOpts.Storage + resourceOpts.Volumes.Total(), + }, + Usage: nil, + } + + return resourceInfo, v.doSetNodeResourceInfo(ctx, node, resourceInfo) +} + +// RemoveNode . +func (v *Volume) RemoveNode(ctx context.Context, node string) error { + if _, err := v.store.Delete(ctx, fmt.Sprintf(NodeResourceInfoKey, node)); err != nil { + logrus.Errorf("[doSetNodeResourceInfo] faield to delete node %v, err: %v", node, err) + return err + } + return nil +} diff --git a/resources/volume/models/node_test.go b/resources/volume/models/node_test.go new file mode 100644 index 000000000..5b6300505 --- /dev/null +++ b/resources/volume/models/node_test.go @@ -0,0 +1,48 @@ +package models + +import ( + "context" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/volume/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestAddNode(t *testing.T) { + ctx := context.Background() + + volume := newTestVolume(t) + nodes := generateNodes(t, volume, 1) + + resourceOpts := &types.NodeResourceOpts{ + Volumes: types.VolumeMap{"/data0": units.TiB}, + Storage: units.TiB, + } + + // existent node + _, err := volume.AddNode(ctx, nodes[0], resourceOpts) + assert.ErrorIs(t, err, types.ErrNodeExists) + + // normal case + resourceInfo, err := volume.AddNode(ctx, "new-node", resourceOpts) + assert.NoError(t, err) + assert.Equal(t, resourceInfo.Capacity.Storage, resourceOpts.Storage+resourceOpts.Volumes.Total()) + assert.Equal(t, resourceInfo.Capacity.Volumes, resourceOpts.Volumes) + assert.Equal(t, resourceInfo.Usage, &types.NodeResourceArgs{Volumes: types.VolumeMap{"/data0": 0}}) +} + +func TestRemoveNode(t *testing.T) { + ctx := context.Background() + + volume := newTestVolume(t) + nodes := generateNodes(t, volume, 1) + + assert.Nil(t, volume.RemoveNode(ctx, nodes[0])) + _, _, err := volume.GetNodeResourceInfo(ctx, nodes[0], nil, false) + assert.ErrorIs(t, err, coretypes.ErrBadCount) + + assert.Nil(t, volume.RemoveNode(ctx, "invalid-node")) +} diff --git a/resources/volume/models/realloc.go b/resources/volume/models/realloc.go new file mode 100644 index 000000000..38ceccef1 --- /dev/null +++ b/resources/volume/models/realloc.go @@ -0,0 +1,89 @@ +package models + +import ( + "context" + + "github.com/sanity-io/litter" + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/volume/schedule" + "github.com/projecteru2/core/resources/volume/types" +) + +// GetReallocArgs . +func (v *Volume) GetReallocArgs(ctx context.Context, node string, originResourceArgs *types.WorkloadResourceArgs, resourceOpts *types.WorkloadResourceOpts) (*types.EngineArgs, *types.WorkloadResourceArgs, *types.WorkloadResourceArgs, error) { + resourceInfo, err := v.doGetNodeResourceInfo(ctx, node) + if err != nil { + logrus.Errorf("[Realloc] failed to get resource info of node %v, err: %v", node, err) + return nil, nil, nil, err + } + + resourceOpts = &types.WorkloadResourceOpts{ + VolumesRequest: types.MergeVolumeBindings(resourceOpts.VolumesRequest, originResourceArgs.VolumesRequest), + VolumesLimit: types.MergeVolumeBindings(resourceOpts.VolumesLimit, originResourceArgs.VolumesLimit), + StorageRequest: resourceOpts.StorageRequest + originResourceArgs.StorageRequest, + StorageLimit: resourceOpts.StorageLimit + originResourceArgs.StorageLimit, + } + + if err := resourceOpts.Validate(); err != nil { + logrus.Errorf("[Realloc] invalid resource opts %v, err: %v", litter.Sdump(resourceOpts), err) + return nil, nil, nil, err + } + + finalWorkloadResourceArgs := &types.WorkloadResourceArgs{ + VolumesRequest: resourceOpts.VolumesRequest, + VolumesLimit: resourceOpts.VolumesLimit, + VolumePlanRequest: nil, + VolumePlanLimit: nil, + StorageRequest: resourceOpts.StorageRequest, + StorageLimit: resourceOpts.StorageLimit, + } + + if finalWorkloadResourceArgs.StorageRequest > resourceInfo.Capacity.Storage-resourceInfo.Usage.Storage { + return nil, nil, nil, types.ErrInsufficientResource + } + + volumePlan := schedule.GetAffinityPlan(resourceInfo, resourceOpts.VolumesRequest, originResourceArgs.VolumePlanRequest) + if volumePlan == nil { + return nil, nil, nil, types.ErrInsufficientResource + } + + finalWorkloadResourceArgs.VolumePlanRequest = volumePlan + finalWorkloadResourceArgs.VolumePlanLimit = getVolumePlanLimit(finalWorkloadResourceArgs.VolumesRequest, finalWorkloadResourceArgs.VolumesLimit, volumePlan) + + originBindingSet := map[[3]string]struct{}{} + for binding := range originResourceArgs.VolumePlanLimit { + originBindingSet[binding.GetMapKey()] = struct{}{} + } + + engineArgs := &types.EngineArgs{Storage: finalWorkloadResourceArgs.StorageLimit} + for _, binding := range resourceOpts.VolumesLimit.ApplyPlan(volumePlan) { + engineArgs.Volumes = append(engineArgs.Volumes, binding.ToString(true)) + if _, ok := originBindingSet[binding.GetMapKey()]; !ok { + engineArgs.VolumeChanged = true + } + } + + deltaWorkloadResourceArgs := getDeltaWorkloadResourceArgs(originResourceArgs, finalWorkloadResourceArgs) + return engineArgs, deltaWorkloadResourceArgs, finalWorkloadResourceArgs, nil +} + +func getDeltaWorkloadResourceArgs(originWorkloadResourceArgs, finalWorkloadResourceArgs *types.WorkloadResourceArgs) *types.WorkloadResourceArgs { + deltaVolumeMap := types.VolumeMap{} + for _, volumeMap := range finalWorkloadResourceArgs.VolumePlanRequest { + deltaVolumeMap.Add(volumeMap) + } + for _, volumeMap := range originWorkloadResourceArgs.VolumePlanRequest { + deltaVolumeMap.Sub(volumeMap) + } + + return &types.WorkloadResourceArgs{ + VolumePlanRequest: types.VolumePlan{&types.VolumeBinding{ + Source: "fake-source", + Destination: "fake-destination", + Flags: "fake-flags", + SizeInBytes: 0, + }: deltaVolumeMap}, + StorageRequest: finalWorkloadResourceArgs.StorageRequest - originWorkloadResourceArgs.StorageRequest, + } +} diff --git a/resources/volume/models/realloc_test.go b/resources/volume/models/realloc_test.go new file mode 100644 index 000000000..b11cef460 --- /dev/null +++ b/resources/volume/models/realloc_test.go @@ -0,0 +1,127 @@ +package models + +import ( + "context" + "testing" + + "github.com/docker/go-units" + "github.com/sanity-io/litter" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/volume/types" + coretypes "github.com/projecteru2/core/types" +) + +func TestRealloc(t *testing.T) { + ctx := context.Background() + + volume := newTestVolume(t) + nodes := generateNodes(t, volume, 1) + node := nodes[0] + + plan := types.VolumePlan{} + err := plan.UnmarshalJSON([]byte(` +{ + "AUTO:/dir0:rw:100GiB": { + "/data0": 107374182400 + }, + "AUTO:/dir1:mrw:100GiB": { + "/data2": 1099511627776 + }, + "AUTO:/dir2:rw:0": { + "/data0": 0 + } +} +`)) + assert.Nil(t, err) + bindings := generateVolumeBindings(t, []string{ + "AUTO:/dir0:rw:100GiB", + "AUTO:/dir1:mrw:100GiB", + "AUTO:/dir2:rw:0", + }) + + originResourceArgs := &types.WorkloadResourceArgs{ + VolumesRequest: bindings, + VolumesLimit: bindings, + VolumePlanRequest: plan, + VolumePlanLimit: plan, + StorageRequest: 0, + StorageLimit: 0, + } + + _, _, err = volume.SetNodeResourceUsage(ctx, node, nil, nil, []*types.WorkloadResourceArgs{originResourceArgs}, true, true) + assert.Nil(t, err) + + // non-existent node + _, _, _, err = volume.GetReallocArgs(ctx, "invalid-node", originResourceArgs, &types.WorkloadResourceOpts{}) + assert.ErrorIs(t, err, coretypes.ErrBadCount) + + // invalid resource opts + opts := &types.WorkloadResourceOpts{ + VolumesRequest: generateVolumeBindings(t, []string{ + "AUTO:/dir0:rw:100GiB", + "AUTO:/dir1:mrw:100GiB", + "AUTO:/dir2:rw:0", + }), + VolumesLimit: nil, + StorageRequest: -1, + StorageLimit: -1, + } + _, _, _, err = volume.GetReallocArgs(ctx, node, originResourceArgs, opts) + assert.ErrorIs(t, err, types.ErrInvalidStorage) + + // insufficient storage + bindings = generateVolumeBindings(t, []string{ + "AUTO:/dir1:mrw:100GiB", + }) + opts = &types.WorkloadResourceOpts{ + VolumesRequest: bindings, + VolumesLimit: bindings, + StorageRequest: 4 * units.TiB, + StorageLimit: 4 * units.TiB, + } + _, _, _, err = volume.GetReallocArgs(ctx, node, originResourceArgs, opts) + assert.ErrorIs(t, err, types.ErrInsufficientResource) + + // insufficient volume + bindings = generateVolumeBindings(t, []string{ + "AUTO:/dir1:mrw:1TiB", + }) + opts = &types.WorkloadResourceOpts{ + VolumesRequest: bindings, + VolumesLimit: bindings, + StorageRequest: 0, + StorageLimit: 0, + } + _, _, _, err = volume.GetReallocArgs(ctx, node, originResourceArgs, opts) + assert.ErrorIs(t, err, types.ErrInsufficientResource) + + // normal case + bindings = generateVolumeBindings(t, []string{ + "AUTO:/dir1:mrw:100GiB", + }) + opts = &types.WorkloadResourceOpts{ + VolumesRequest: bindings, + VolumesLimit: bindings, + StorageRequest: units.GiB, + StorageLimit: units.GiB, + } + + _, _, finalWorkloadResourceArgs, err := volume.GetReallocArgs(ctx, node, originResourceArgs, opts) + assert.Nil(t, err) + plan = types.VolumePlan{} + assert.Nil(t, plan.UnmarshalJSON([]byte(` +{ + "AUTO:/dir0:rw:100GiB": { + "/data0": 107374182400 + }, + "AUTO:/dir1:mrw:200GiB": { + "/data2": 1099511627776 + }, + "AUTO:/dir2:rw:0": { + "/data0": 0 + } +} +`))) + assert.Equal(t, litter.Sdump(plan), litter.Sdump(finalWorkloadResourceArgs.VolumePlanRequest)) +} diff --git a/resources/volume/models/remap.go b/resources/volume/models/remap.go new file mode 100644 index 000000000..f93904025 --- /dev/null +++ b/resources/volume/models/remap.go @@ -0,0 +1,12 @@ +package models + +import ( + "context" + + "github.com/projecteru2/core/resources/volume/types" +) + +// GetRemapArgs . +func (v *Volume) GetRemapArgs(ctx context.Context, node string, workloadResourceMap *types.WorkloadResourceArgsMap) (map[string]*types.EngineArgs, error) { + return nil, nil +} diff --git a/resources/volume/models/volume.go b/resources/volume/models/volume.go new file mode 100644 index 000000000..3782054c5 --- /dev/null +++ b/resources/volume/models/volume.go @@ -0,0 +1,28 @@ +package models + +import ( + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/store/etcdv3/meta" + coretypes "github.com/projecteru2/core/types" +) + +// Volume . +type Volume struct { + Config coretypes.Config + store meta.KV +} + +// NewVolume . +func NewVolume(config coretypes.Config) (*Volume, error) { + v := &Volume{Config: config} + var err error + if len(config.Etcd.Machines) > 0 { + v.store, err = meta.NewETCD(config.Etcd, nil) + if err != nil { + logrus.Errorf("[NewVolume] failed to create etcd client, err: %v", err) + return nil, err + } + } + return v, nil +} diff --git a/resources/volume/models/volume_test.go b/resources/volume/models/volume_test.go new file mode 100644 index 000000000..fc7fb7a2f --- /dev/null +++ b/resources/volume/models/volume_test.go @@ -0,0 +1,78 @@ +package models + +import ( + "context" + "fmt" + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/volume/types" + "github.com/projecteru2/core/store/etcdv3/meta" + coretypes "github.com/projecteru2/core/types" +) + +func newTestVolume(t *testing.T) *Volume { + config := coretypes.Config{ + Etcd: coretypes.EtcdConfig{ + Prefix: "/Volume", + }, + Scheduler: coretypes.SchedConfig{ + MaxShare: -1, + ShareBase: 100, + MaxDeployCount: 1000, + }, + } + volume, err := NewVolume(config) + assert.Nil(t, err) + store, err := meta.NewETCD(config.Etcd, t) + assert.Nil(t, err) + volume.store = store + return volume +} + +func generateNodeResourceInfos(num int) []*types.NodeResourceInfo { + var infos []*types.NodeResourceInfo + for i := 0; i < num; i++ { + info := &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + Volumes: types.VolumeMap{ + "/data0": units.TiB, + "/data1": units.TiB, + "/data2": units.TiB, + "/data3": units.TiB, + }, + Storage: 4 * units.TiB, + }, + Usage: &types.NodeResourceArgs{ + Volumes: types.VolumeMap{ + "/data0": 200 * units.GiB, + "/data1": 300 * units.GiB, + }, + Storage: 500 * units.GiB, + }, + } + infos = append(infos, info) + } + return infos +} + +func generateNodes(t *testing.T, volume *Volume, num int) []string { + nodes := []string{} + infos := generateNodeResourceInfos(num) + + for i, info := range infos { + nodeName := fmt.Sprintf("node%d", i) + assert.Nil(t, volume.doSetNodeResourceInfo(context.Background(), nodeName, info)) + nodes = append(nodes, nodeName) + } + + return nodes +} + +func generateVolumeBindings(t *testing.T, str []string) types.VolumeBindings { + bindings, err := types.NewVolumeBindings(str) + assert.Nil(t, err) + return bindings +} diff --git a/resources/volume/schedule/schedule.go b/resources/volume/schedule/schedule.go new file mode 100644 index 000000000..f4409c9c3 --- /dev/null +++ b/resources/volume/schedule/schedule.go @@ -0,0 +1,454 @@ +package schedule + +import ( + "container/heap" + "math" + "sort" + + "github.com/sirupsen/logrus" + + "github.com/projecteru2/core/resources/volume/types" +) + +type volume struct { + device string + size int64 +} + +func (v *volume) LessThan(v1 *volume) bool { + if v.size == v1.size { + return v.device < v1.device + } + return v.size < v1.size +} + +type volumes []*volume + +// DeepCopy . +func (v volumes) DeepCopy() volumes { + res := volumes{} + for _, item := range v { + res = append(res, &volume{device: item.device, size: item.size}) + } + return res +} + +type volumeHeap volumes + +// Len . +func (v volumeHeap) Len() int { + return len(v) +} + +// Less . +func (v volumeHeap) Less(i, j int) bool { + return v[i].LessThan(v[j]) +} + +// Swap . +func (v volumeHeap) Swap(i, j int) { + v[i], v[j] = v[j], v[i] +} + +// Push . +func (v *volumeHeap) Push(x interface{}) { + *v = append(*v, x.(*volume)) +} + +// Pop . +func (v *volumeHeap) Pop() interface{} { + old := *v + n := len(old) + x := old[n-1] + *v = old[:n-1] + return x +} + +func min(a, b int) int { + if a < b { + return a + } + return b +} + +type host struct { + maxDeployCount int + usedVolumes volumes + unusedVolumes volumes +} + +func newHost(resourceInfo *types.NodeResourceInfo, maxDeployCount int) *host { + h := &host{ + maxDeployCount: maxDeployCount, + usedVolumes: []*volume{}, + unusedVolumes: []*volume{}, + } + + for device, size := range resourceInfo.Capacity.Volumes { + used := resourceInfo.Usage.Volumes[device] + if used == 0 { + h.unusedVolumes = append(h.unusedVolumes, &volume{device: device, size: size}) + } else { + h.usedVolumes = append(h.usedVolumes, &volume{device: device, size: size - used}) + } + } + + sort.SliceStable(h.unusedVolumes, func(i, j int) bool { return h.unusedVolumes[i].LessThan(h.unusedVolumes[j]) }) + sort.SliceStable(h.usedVolumes, func(i, j int) bool { return h.usedVolumes[i].LessThan(h.usedVolumes[j]) }) + return h +} + +func (h *host) getMonoPlan(monoRequests types.VolumeBindings, totalRequestSize int64, volume *volume) types.VolumePlan { + if volume.size < totalRequestSize { + return nil + } + volumePlan := types.VolumePlan{} + + volumeSize := volume.size + for _, req := range monoRequests { + size := int64(float64(req.SizeInBytes) / float64(totalRequestSize) * float64(volumeSize)) + volumePlan[req] = types.VolumeMap{volume.device: size} + volume.size -= size + } + + if volume.size != 0 { + for _, volumeMap := range volumePlan { + volumeMap[volumeMap.GetDevice()] += volume.size + break + } + } + + return volumePlan +} + +func (h *host) getMonoPlans(monoRequests types.VolumeBindings) ([]types.VolumePlan, int) { + if len(monoRequests) == 0 { + return []types.VolumePlan{}, h.maxDeployCount + } + if len(h.unusedVolumes) == 0 { + return []types.VolumePlan{}, 0 + } + + totalSize := int64(0) + for _, req := range monoRequests { + totalSize += req.SizeInBytes + } + + volumes := h.unusedVolumes.DeepCopy() + volumePlans := []types.VolumePlan{} + + // h.unusedVolumes have already been sorted + for _, volume := range volumes { + if volumePlan := h.getMonoPlan(monoRequests, totalSize, volume); volumePlan != nil { + volumePlans = append(volumePlans, volumePlan) + } + } + + return volumePlans, len(volumePlans) +} + +func (h *host) getNormalPlan(vHeap *volumeHeap, normalRequests types.VolumeBindings) types.VolumePlan { + volumePlan := types.VolumePlan{} + for reqIndex := 0; reqIndex < len(normalRequests); reqIndex++ { + req := normalRequests[reqIndex] + volumeToPush := []*volume{} + allocated := false + + for vHeap.Len() > 0 { + volume := heap.Pop(vHeap).(*volume) + if volume.size >= req.SizeInBytes { + volumePlan[req] = types.VolumeMap{volume.device: req.SizeInBytes} + allocated = true + volume.size -= req.SizeInBytes + if volume.size > 0 { + volumeToPush = append(volumeToPush, volume) + } + break + } + } + + for _, volume := range volumeToPush { + heap.Push(vHeap, volume) + } + + if !allocated { + return nil + } + } + + return volumePlan +} + +func (h *host) getNormalPlans(normalRequests types.VolumeBindings) ([]types.VolumePlan, int) { + if len(normalRequests) == 0 { + return []types.VolumePlan{}, h.maxDeployCount + } + + vh := volumeHeap(h.usedVolumes.DeepCopy()) + vHeap := &vh + heap.Init(vHeap) + + volumePlans := []types.VolumePlan{} + + for len(volumePlans) <= h.maxDeployCount { + if volumePlan := h.getNormalPlan(vHeap, normalRequests); volumePlan != nil { + volumePlans = append(volumePlans, volumePlan) + } else { + break + } + } + return volumePlans, len(volumePlans) +} + +func (h *host) getUnlimitedPlans(normalPlans, monoPlans []types.VolumePlan, unlimitedRequests types.VolumeBindings) []types.VolumePlan { + capacity := len(normalPlans) + + volumes := append(h.usedVolumes.DeepCopy(), h.unusedVolumes.DeepCopy()...) + volumeMap := map[string]*volume{} + for _, volume := range volumes { + volumeMap[volume.device] = volume + } + + // apply normal changes + for _, plan := range normalPlans { + for _, vm := range plan { + for device, size := range vm { + volumeMap[device].size -= size + } + } + } + + // apply mono changes + for _, plan := range monoPlans { + for _, vm := range plan { + for device, size := range vm { + volumeMap[device].size -= size + } + } + } + + // select the volume with the largest size + maxVolume := volumes[0] + for i := range volumes { + if volumes[i].size > maxVolume.size { + maxVolume = volumes[i] + } + } + + plans := []types.VolumePlan{} + for i := 0; i < capacity; i++ { + plan := types.VolumePlan{} + for _, req := range unlimitedRequests { + plan[req] = types.VolumeMap{maxVolume.device: req.SizeInBytes} + } + plans = append(plans, plan) + } + + return plans +} + +func (h *host) getVolumePlans(volumeBindings types.VolumeBindings) []types.VolumePlan { + if len(h.unusedVolumes) == 0 && len(h.usedVolumes) == 0 { + return nil + } + + normalRequests, monoRequests, unlimitedRequests := h.classifyVolumeBindings(volumeBindings) + + minNormalRequestSize := int64(math.MaxInt) + for _, normalRequest := range normalRequests { + if normalRequest.SizeInBytes < minNormalRequestSize { + minNormalRequestSize = normalRequest.SizeInBytes + } + } + + // get baseline + normalPlans, normalCapacity := h.getNormalPlans(normalRequests) + monoPlans, monoCapacity := h.getMonoPlans(monoRequests) + bestCapacity := min(monoCapacity, normalCapacity) + bestVolumePlans := [2][]types.VolumePlan{normalPlans[:min(bestCapacity, len(normalPlans))], monoPlans[:min(bestCapacity, len(monoPlans))]} + + for monoCapacity > normalCapacity { + // convert an unused volume to used volume + p := sort.Search(len(h.unusedVolumes), func(i int) bool { return h.unusedVolumes[i].size >= minNormalRequestSize }) + // if no volume meets the requirement, just stop + if p == len(h.unusedVolumes) { + break + } + v := h.unusedVolumes[p] + h.unusedVolumes = append(h.unusedVolumes[:p], h.unusedVolumes[p+1:]...) + h.usedVolumes = append(h.usedVolumes, v) + + normalPlans, normalCapacity = h.getNormalPlans(normalRequests) + monoPlans, monoCapacity = h.getMonoPlans(monoRequests) + capacity := min(monoCapacity, normalCapacity) + if capacity > bestCapacity { + bestCapacity = capacity + bestVolumePlans = [2][]types.VolumePlan{normalPlans[:min(len(normalPlans), capacity)], monoPlans[:min(len(monoPlans), capacity)]} + } + } + + normalPlans, monoPlans = bestVolumePlans[0], bestVolumePlans[1] + unlimitedPlans := h.getUnlimitedPlans(normalPlans, monoPlans, unlimitedRequests) + + resVolumePlans := []types.VolumePlan{} + merge := func(plan types.VolumePlan, plans []types.VolumePlan, index int) { + if index < len(plans) { + plan.Merge(plans[index]) + } + } + + for i := 0; i < bestCapacity; i++ { + plan := types.VolumePlan{} + merge(plan, normalPlans, i) + merge(plan, monoPlans, i) + merge(plan, unlimitedPlans, i) + resVolumePlans = append(resVolumePlans, plan) + } + return resVolumePlans +} + +func (h *host) classifyVolumeBindings(volumeBindings types.VolumeBindings) (normalRequests, monoRequests, unlimitedRequests types.VolumeBindings) { + for _, binding := range volumeBindings { + switch { + case binding.RequireScheduleMonopoly(): + monoRequests = append(monoRequests, binding) + case binding.RequireScheduleUnlimitedQuota(): + unlimitedRequests = append(unlimitedRequests, binding) + case binding.RequireSchedule(): + normalRequests = append(normalRequests, binding) + } + } + + sort.SliceStable(monoRequests, func(i, j int) bool { return monoRequests[i].SizeInBytes < monoRequests[j].SizeInBytes }) + sort.SliceStable(normalRequests, func(i, j int) bool { return normalRequests[i].SizeInBytes < normalRequests[j].SizeInBytes }) + + return normalRequests, monoRequests, unlimitedRequests +} + +func (h *host) classifyAffinityRequests(requests types.VolumeBindings, existing types.VolumePlan) (affinity map[*types.VolumeBinding]types.VolumeMap, nonAffinity map[*types.VolumeBinding]types.VolumeMap) { + affinity = map[*types.VolumeBinding]types.VolumeMap{} + nonAffinity = map[*types.VolumeBinding]types.VolumeMap{} + + for _, req := range requests { + found := false + for binding, volumeMap := range existing { + if req.Source == binding.Source && req.Destination == binding.Destination && req.Flags == binding.Flags { + affinity[req] = volumeMap + found = true + break + } + } + if !found { + nonAffinity[req] = nil + } + } + return affinity, nonAffinity +} + +func (h *host) getVolumeByDevice(device string) *volume { + for _, volume := range h.usedVolumes { + if volume.device == device { + return volume + } + } + for _, volume := range h.unusedVolumes { + if volume.device == device { + return volume + } + } + return nil +} + +func (h *host) getAffinityPlan(volumeRequest types.VolumeBindings, existing types.VolumePlan) types.VolumePlan { + normalRequests, monoRequests, unlimitedRequests := h.classifyVolumeBindings(volumeRequest) + needRescheduleRequests := types.VolumeBindings{} + volumePlan := types.VolumePlan{} + + commonProcess := func(requests types.VolumeBindings) error { + affinity, nonAffinity := h.classifyAffinityRequests(requests, existing) + for binding, volumeMap := range affinity { + device := volumeMap.GetDevice() + size := volumeMap.GetSize() + // check if the device has enough space + diff := binding.SizeInBytes - size + if diff > h.getVolumeByDevice(device).size { + logrus.Errorf("[getAffinityPlan] no space to expand, %v remains %v, requires %v", device, h.getVolumeByDevice(device).size, diff) + return types.ErrInsufficientResource + } + volumePlan.Merge(types.VolumePlan{binding: types.VolumeMap{device: binding.SizeInBytes}}) + } + for binding := range nonAffinity { + needRescheduleRequests = append(needRescheduleRequests, binding) + } + return nil + } + + // normal + if err := commonProcess(normalRequests); err != nil { + return nil + } + + totalRequestSize := int64(0) + totalVolumeSize := int64(0) + for _, binding := range monoRequests { + totalRequestSize += binding.SizeInBytes + } + for binding, volumeMap := range existing { + if binding.RequireScheduleMonopoly() { + totalVolumeSize += volumeMap.GetSize() + } + } + + affinity, nonAffinity := h.classifyAffinityRequests(monoRequests, existing) + // if there is no affinity plan: all reschedule + if len(affinity) == 0 { + for binding := range nonAffinity { + needRescheduleRequests = append(needRescheduleRequests, binding) + } + } else { + // if there is any affinity plan: don't reschedule + // use the first volume map to get the whole mono volume plan + if totalVolumeSize < totalRequestSize { + logrus.Errorf("[getAffinityPlan] no space to expand, the size of %v is %v, requires %v", affinity[monoRequests[0]].GetDevice(), totalVolumeSize, totalRequestSize) + return nil + } + for _, volumeMap := range affinity { + for _, volume := range h.usedVolumes { + if volume.device == volumeMap.GetDevice() { + volume.size = totalVolumeSize + volumePlan.Merge(h.getMonoPlan(monoRequests, totalRequestSize, volume)) + break + } + } + break + } + } + + // unlimited + if err := commonProcess(unlimitedRequests); err != nil { + return nil + } + + if len(needRescheduleRequests) == 0 { + return volumePlan + } + + volumePlans := h.getVolumePlans(needRescheduleRequests) + if len(volumePlans) == 0 { + return nil + } + volumePlan.Merge(volumePlans[0]) + return volumePlan +} + +// GetAffinityPlan . +func GetAffinityPlan(resourceInfo *types.NodeResourceInfo, volumeRequest types.VolumeBindings, existing types.VolumePlan) types.VolumePlan { + h := newHost(resourceInfo, 1) + return h.getAffinityPlan(volumeRequest, existing) +} + +// GetVolumePlans . +func GetVolumePlans(resourceInfo *types.NodeResourceInfo, volumeRequest types.VolumeBindings, maxDeployCount int) []types.VolumePlan { + h := newHost(resourceInfo, maxDeployCount) + return h.getVolumePlans(volumeRequest) +} diff --git a/resources/volume/schedule/schedule_test.go b/resources/volume/schedule/schedule_test.go new file mode 100644 index 000000000..9aa1db6fa --- /dev/null +++ b/resources/volume/schedule/schedule_test.go @@ -0,0 +1,306 @@ +package schedule + +import ( + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" + + "github.com/projecteru2/core/resources/volume/types" +) + +var maxDeployCount = 1000 + +func generateResourceInfo() *types.NodeResourceInfo { + return &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + Volumes: types.VolumeMap{ + "/data0": units.TiB, + "/data1": units.TiB, + "/data2": units.TiB, + "/data3": units.TiB, + }, + }, + Usage: &types.NodeResourceArgs{ + Volumes: types.VolumeMap{ + "/data0": 200 * units.GiB, + "/data1": 300 * units.GiB, + }, + }, + } +} + +func generateEmptyResourceInfo() *types.NodeResourceInfo { + return &types.NodeResourceInfo{ + Capacity: &types.NodeResourceArgs{ + Volumes: types.VolumeMap{}, + }, + Usage: &types.NodeResourceArgs{ + Volumes: types.VolumeMap{}, + }, + } +} + +func applyPlans(resourceInfo *types.NodeResourceInfo, plans []types.VolumePlan) { + for _, plan := range plans { + for _, volumeMap := range plan { + for device, size := range volumeMap { + resourceInfo.Usage.Volumes[device] += size + } + } + } +} + +func noMorePlans(t *testing.T, resourceInfo *types.NodeResourceInfo, volumePlans []types.VolumePlan, volumeRequest types.VolumeBindings) { + applyPlans(resourceInfo, volumePlans) + assert.Nil(t, resourceInfo.Validate()) + plan := GetVolumePlans(resourceInfo, volumeRequest, maxDeployCount) + assert.Equal(t, len(plan), 0) +} + +func validateVolumePlan(t *testing.T, resourceInfo *types.NodeResourceInfo, volumeRequest types.VolumeBindings, volumePlan types.VolumePlan) { + t.Logf("volume plan: %v", volumePlan) + t.Logf("volume request: %v", volumeRequest) + monoDevice := "" + monoTotalSize := int64(0) + for _, binding := range volumeRequest { + if !binding.RequireSchedule() { + continue + } + volumeMap, ok := volumePlan[binding] + assert.True(t, ok) + switch { + case binding.RequireScheduleMonopoly(): + if monoDevice == "" { + monoDevice = volumeMap.GetDevice() + } + assert.Equal(t, monoDevice, volumeMap.GetDevice()) + monoTotalSize += volumeMap.GetSize() + case binding.RequireSchedule(): + assert.Equal(t, volumeMap.GetSize(), binding.SizeInBytes) + } + } + + assert.Equal(t, monoTotalSize, resourceInfo.Capacity.Volumes[monoDevice]) +} + +func validateVolumePlans(t *testing.T, resourceInfo *types.NodeResourceInfo, volumeRequest types.VolumeBindings, volumePlans []types.VolumePlan) { + t.Logf("%v plans in total", len(volumePlans)) + t.Logf("plans: %v", volumePlans) + for _, plan := range volumePlans { + validateVolumePlan(t, resourceInfo, volumeRequest, plan) + } + noMorePlans(t, resourceInfo, volumePlans, volumeRequest) +} + +func generateVolumeBindings(t *testing.T, str []string) types.VolumeBindings { + bindings, err := types.NewVolumeBindings(str) + assert.Nil(t, err) + return bindings +} + +func TestGetVolumePlans(t *testing.T) { + // no volume + resourceInfo := generateEmptyResourceInfo() + + // single normal request + volumeRequest, err := types.NewVolumeBindings([]string{}) + assert.Nil(t, err) + + plans := GetVolumePlans(resourceInfo, volumeRequest, maxDeployCount) + assert.Equal(t, len(plans), 0) + + // normal cases + requests := []types.VolumeBindings{ + // single normal request + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rw:500GiB", + }), + // multiple normal request + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rw:500GiB", + "AUTO:/dir2:rw:500GiB", + }), + // with single mono request + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rw:1GiB", + "AUTO:/dir2:rwm:1GiB", + }), + // with mono requests + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rw:500GiB", + "AUTO:/dir2:rw:500GiB", + "AUTO:/dir3:rwm:100GiB", + "AUTO:/dir4:rwm:100GiB", + "AUTO:/dir5:rwm:100GiB", + }), + // with unlimited requests + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rw:500GiB", + "AUTO:/dir2:rw:0", + "AUTO:/dir3:rwm:100GiB", + "AUTO:/dir4:rwm:100GiB", + "AUTO:/dir5:rwm:100GiB", + }), + } + + for _, volumeRequest := range requests { + resourceInfo = generateResourceInfo() + plans = GetVolumePlans(resourceInfo, volumeRequest, maxDeployCount) + validateVolumePlans(t, resourceInfo, volumeRequest, plans) + } + + // invalid requests + requests = []types.VolumeBindings{ + // single normal request with too much size + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rw:2TiB", + }), + // multiple normal request with too much size + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rw:800GiB", + "AUTO:/dir2:rw:800GiB", + "AUTO:/dir3:rw:800GiB", + "AUTO:/dir4:rw:800GiB", + "AUTO:/dir5:rw:800GiB", + "AUTO:/dir6:rw:800GiB", + "AUTO:/dir7:rw:800GiB", + }), + // mono request with too much size + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rwm:500GiB", + "AUTO:/dir2:rwm:500GiB", + "AUTO:/dir3:rwm:500GiB", + }), + // insufficient unused volume + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rw:800GiB", + "AUTO:/dir2:rw:800GiB", + "AUTO:/dir3:rw:800GiB", + "AUTO:/dir4:rw:800GiB", + "AUTO:/dir5:rwm:500GiB", + "AUTO:/dir6:rwm:500GiB", + }), + } + + for _, volumeRequest := range requests { + resourceInfo = generateResourceInfo() + plans = GetVolumePlans(resourceInfo, volumeRequest, maxDeployCount) + assert.Equal(t, len(plans), 0) + } +} + +func generateExistingVolumePlan(t *testing.T) (types.VolumeBindings, types.VolumePlan) { + plan := types.VolumePlan{} + err := plan.UnmarshalJSON([]byte(` +{ + "AUTO:/dir0:rw:100GiB": { + "/data0": 107374182400 + }, + "AUTO:/dir1:mrw:100GiB": { + "/data2": 1099511627776 + }, + "AUTO:/dir2:rw:0": { + "/data0": 0 + } +} +`)) + assert.Nil(t, err) + bindings := generateVolumeBindings(t, []string{ + "AUTO:/dir0:rw:100GiB", + "AUTO:/dir1:mrw:100GiB", + "AUTO:/dir2:rw:0", + }) + return bindings, plan +} + +func TestGetAffinityPlan(t *testing.T) { + // normal cases + requests := []types.VolumeBindings{ + // realloc normal + generateVolumeBindings(t, []string{ + "AUTO:/dir0:rw:1GiB", + }), + // realloc normal with reschedule + generateVolumeBindings(t, []string{ + "AUTO:/dir3:rw:1GiB", + }), + // realloc mono without reschedule + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rwm:1GiB", + }), + // realloc mono without reschedule + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rwm:1GiB", + "AUTO:/dir3:rwm:1GiB", + }), + // realloc mono with reschedule + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rwm:-1TiB", + "AUTO:/dir3:rwm:100GiB", + "AUTO:/dir4:rwm:100GiB", + }), + // realloc unlimited + generateVolumeBindings(t, []string{ + "AUTO:/dir0:rw:-100GiB", + }), + // mixed + generateVolumeBindings(t, []string{ + "AUTO:/dir0:rw:-100GiB", + "AUTO:/dir2:rw:100GiB", + "AUTO:/dir3:rwm:100GiB", + }), + } + + for _, request := range requests { + resourceInfo := generateResourceInfo() + originRequest, existing := generateExistingVolumePlan(t) + for _, volumeMap := range existing { + resourceInfo.Usage.Volumes[volumeMap.GetDevice()] += volumeMap.GetSize() + } + mergedRequest := types.MergeVolumeBindings(request, originRequest) + + plan := GetAffinityPlan(resourceInfo, mergedRequest, existing) + validateVolumePlan(t, resourceInfo, mergedRequest, plan) + } + + // no request + resourceInfo := generateResourceInfo() + originRequest, existing := generateExistingVolumePlan(t) + for _, volumeMap := range existing { + resourceInfo.Usage.Volumes[volumeMap.GetDevice()] += volumeMap.GetSize() + } + emptyRequest := types.VolumeBindings{} + mergedRequest := types.MergeVolumeBindings(emptyRequest, originRequest) + + plan := GetAffinityPlan(resourceInfo, mergedRequest, existing) + assert.Equal(t, existing.String(), plan.String()) + + invalidRequests := []types.VolumeBindings{ + // normal request with too much size + generateVolumeBindings(t, []string{ + "AUTO:/dir0:rw:1TiB", + }), + // mono request with too much size + generateVolumeBindings(t, []string{ + "AUTO:/dir1:rwm:1TiB", + }), + // volumes are not enough + generateVolumeBindings(t, []string{ + "AUTO:/dir3:rw:1TiB", + "AUTO:/dir4:rw:1TiB", + }), + } + + for _, request := range invalidRequests { + resourceInfo := generateResourceInfo() + originRequest, existing := generateExistingVolumePlan(t) + for _, volumeMap := range existing { + resourceInfo.Usage.Volumes[volumeMap.GetDevice()] += volumeMap.GetSize() + } + mergedRequest := types.MergeVolumeBindings(request, originRequest) + + plan := GetAffinityPlan(resourceInfo, mergedRequest, existing) + assert.Equal(t, len(plan), 0) + } +} diff --git a/resources/volume/types/errors.go b/resources/volume/types/errors.go new file mode 100644 index 000000000..7ee74a372 --- /dev/null +++ b/resources/volume/types/errors.go @@ -0,0 +1,12 @@ +package types + +import "errors" + +var ( + ErrInvalidCapacity = errors.New("invalid capacity") + ErrInvalidVolume = errors.New("invalid volume") + ErrInsufficientResource = errors.New("cannot alloc a plan, not enough resource") + ErrInvalidStorage = errors.New("invalid storage") + + ErrNodeExists = errors.New("node already exists") +) diff --git a/resources/volume/types/resource.go b/resources/volume/types/resource.go new file mode 100644 index 000000000..fe6b96f62 --- /dev/null +++ b/resources/volume/types/resource.go @@ -0,0 +1,286 @@ +package types + +import ( + "encoding/json" + "sort" + "strings" + "sync" + + "github.com/pkg/errors" + + coretypes "github.com/projecteru2/core/types" + coreutils "github.com/projecteru2/core/utils" +) + +// WorkloadResourceOpts . +type WorkloadResourceOpts struct { + VolumesRequest VolumeBindings `json:"volumes_request"` + VolumesLimit VolumeBindings `json:"volumes_limit"` + + StorageRequest int64 `json:"storage_request"` + StorageLimit int64 `json:"storage_limit"` + + once sync.Once +} + +// Validate . +func (w *WorkloadResourceOpts) Validate() error { + if len(w.VolumesLimit) > 0 && len(w.VolumesRequest) == 0 { + w.VolumesRequest = w.VolumesLimit + } + if len(w.VolumesRequest) != len(w.VolumesLimit) { + return errors.Wrap(ErrInvalidVolume, "different length of request and limit") + } + + sortFunc := func(volumeBindings []*VolumeBinding) func(i, j int) bool { + return func(i, j int) bool { + return volumeBindings[i].ToString(false) < volumeBindings[j].ToString(false) + } + } + + sort.Slice(w.VolumesRequest, sortFunc(w.VolumesRequest)) + sort.Slice(w.VolumesLimit, sortFunc(w.VolumesLimit)) + + for i := range w.VolumesRequest { + request := w.VolumesRequest[i] + limit := w.VolumesLimit[i] + if request.Source != limit.Source || request.Destination != limit.Destination || request.Flags != limit.Flags { + return errors.Wrap(ErrInvalidVolume, "request and limit not match") + } + if request.SizeInBytes > 0 && limit.SizeInBytes > 0 && request.SizeInBytes > limit.SizeInBytes { + limit.SizeInBytes = request.SizeInBytes + } + } + + if w.StorageLimit < 0 || w.StorageRequest < 0 { + return errors.Wrap(ErrInvalidStorage, "storage limit or request less than 0") + } + if w.StorageLimit > 0 && w.StorageRequest == 0 { + w.StorageRequest = w.StorageLimit + } + if w.StorageLimit > 0 && w.StorageRequest > 0 && w.StorageRequest > w.StorageLimit { + w.StorageLimit = w.StorageRequest // soft limit storage size + } + + // ensure to change storage request / limit only once + w.once.Do(func() { + w.StorageRequest += w.VolumesRequest.TotalSize() + w.StorageLimit += w.VolumesLimit.TotalSize() + }) + return nil +} + +// ParseFromRawParams . +func (w *WorkloadResourceOpts) ParseFromRawParams(rawParams coretypes.RawParams) (err error) { + if w.VolumesRequest, err = NewVolumeBindings(rawParams.OneOfStringSlice("volumes-request", "volume-request", "volumes-request")); err != nil { + return err + } + if w.VolumesLimit, err = NewVolumeBindings(rawParams.OneOfStringSlice("volumes", "volume", "volume-limit", "volumes-limit")); err != nil { + return err + } + + if w.StorageRequest, err = coreutils.ParseRAMInHuman(rawParams.String("storage-request")); err != nil { + return err + } + if w.StorageLimit, err = coreutils.ParseRAMInHuman(rawParams.String("storage-limit")); err != nil { + return err + } + if rawParams.IsSet("storage") { + storage, err := coreutils.ParseRAMInHuman(rawParams.String("storage")) + if err != nil { + return err + } + w.StorageLimit = storage + w.StorageRequest = storage + } + return nil +} + +// WorkloadResourceArgs . +type WorkloadResourceArgs struct { + VolumesRequest VolumeBindings `json:"volumes_request"` + VolumesLimit VolumeBindings `json:"volumes_limit"` + + VolumePlanRequest VolumePlan `json:"volume_plan_request"` + VolumePlanLimit VolumePlan `json:"volume_plan_limit"` + + StorageRequest int64 `json:"storage_request"` + StorageLimit int64 `json:"storage_limit"` +} + +// ParseFromRawParams . +func (w *WorkloadResourceArgs) ParseFromRawParams(rawParams coretypes.RawParams) (err error) { + body, err := json.Marshal(rawParams) + if err != nil { + return err + } + return json.Unmarshal(body, w) +} + +// NodeResourceOpts . +type NodeResourceOpts struct { + Volumes VolumeMap `json:"volumes"` + Storage int64 `json:"storage"` + + RawParams coretypes.RawParams `json:"-"` +} + +// ParseFromRawParams . +func (n *NodeResourceOpts) ParseFromRawParams(rawParams coretypes.RawParams) (err error) { + n.RawParams = rawParams + + volumes := VolumeMap{} + for _, volume := range n.RawParams.StringSlice("volumes") { + parts := strings.Split(volume, ":") + if len(parts) != 2 { + return errors.Wrap(ErrInvalidVolume, "volume should have 2 parts") + } + + capacity, err := coreutils.ParseRAMInHuman(parts[1]) + if err != nil { + return err + } + volumes[parts[0]] = capacity + } + n.Volumes = volumes + + if n.Storage, err = coreutils.ParseRAMInHuman(n.RawParams.String("storage")); err != nil { + return err + } + return nil +} + +// SkipEmpty used for setting node resource capacity in absolute mode +func (n *NodeResourceOpts) SkipEmpty(resourceCapacity *NodeResourceArgs) { + if n == nil { + return + } + if !n.RawParams.IsSet("volumes") { + n.Volumes = resourceCapacity.Volumes + } + if !n.RawParams.IsSet("storage") { + n.Storage = resourceCapacity.Storage + } +} + +// NodeResourceArgs . +type NodeResourceArgs struct { + Volumes VolumeMap `json:"volumes"` + Storage int64 `json:"storage"` +} + +// ParseFromRawParams . +func (n *NodeResourceArgs) ParseFromRawParams(rawParams coretypes.RawParams) error { + body, err := json.Marshal(rawParams) + if err != nil { + return err + } + return json.Unmarshal(body, n) +} + +// DeepCopy . +func (n *NodeResourceArgs) DeepCopy() *NodeResourceArgs { + return &NodeResourceArgs{Volumes: n.Volumes.DeepCopy(), Storage: n.Storage} +} + +// RemoveEmpty . +func (n *NodeResourceArgs) RemoveEmpty() { + for device, size := range n.Volumes { + if n.Volumes[device] == 0 { + n.Storage -= size + delete(n.Volumes, device) + } + } +} + +// Add . +func (n *NodeResourceArgs) Add(n1 *NodeResourceArgs) { + for k, v := range n1.Volumes { + n.Volumes[k] += v + } + n.Storage += n1.Storage +} + +// Sub . +func (n *NodeResourceArgs) Sub(n1 *NodeResourceArgs) { + for k, v := range n1.Volumes { + n.Volumes[k] -= v + } + n.Storage -= n1.Storage +} + +// NodeResourceInfo . +type NodeResourceInfo struct { + Capacity *NodeResourceArgs `json:"capacity"` + Usage *NodeResourceArgs `json:"usage"` +} + +// Validate . +func (n *NodeResourceInfo) Validate() error { + if n.Capacity == nil { + return ErrInvalidCapacity + } + if n.Usage == nil { + n.Usage = &NodeResourceArgs{Volumes: VolumeMap{}, Storage: 0} + for device := range n.Capacity.Volumes { + n.Usage.Volumes[device] = 0 + } + } + + for key, value := range n.Capacity.Volumes { + if value < 0 { + return errors.Wrap(ErrInvalidVolume, "volume size should not be less than 0") + } + if usage, ok := n.Usage.Volumes[key]; ok && (usage > value || usage < 0) { + return errors.Wrap(ErrInvalidVolume, "invalid size in usage") + } + } + for key := range n.Usage.Volumes { + if _, ok := n.Usage.Volumes[key]; !ok { + return errors.Wrap(ErrInvalidVolume, "invalid key in usage") + } + } + + if n.Capacity.Storage < 0 { + return errors.Wrap(ErrInvalidStorage, "storage capacity can't be negative") + } + if n.Usage.Storage < 0 { + return errors.Wrap(ErrInvalidStorage, "storage usage can't be negative") + } + return nil +} + +// GetAvailableResource . +func (n *NodeResourceInfo) GetAvailableResource() *NodeResourceArgs { + res := n.Capacity.DeepCopy() + res.Sub(n.Usage) + return res +} + +// NodeCapacityInfo . +type NodeCapacityInfo struct { + Node string `json:"node"` + Capacity int `json:"capacity"` + Usage float64 `json:"usage"` + Rate float64 `json:"rate"` + Weight int `json:"weight"` +} + +// EngineArgs . +type EngineArgs struct { + Volumes []string `json:"volumes"` + VolumeChanged bool `json:"volume_changed"` // indicates whether the realloc request includes new volumes + Storage int64 `json:"storage"` +} + +// WorkloadResourceArgsMap . +type WorkloadResourceArgsMap map[string]*WorkloadResourceArgs + +// ParseFromRawParamsMap . +func (w *WorkloadResourceArgsMap) ParseFromRawParamsMap(rawParamsMap map[string]coretypes.RawParams) error { + body, err := json.Marshal(rawParamsMap) + if err != nil { + return err + } + return json.Unmarshal(body, w) +} diff --git a/types/volume.go b/resources/volume/types/volume.go similarity index 60% rename from types/volume.go rename to resources/volume/types/volume.go index 016184560..976dfd6fe 100644 --- a/types/volume.go +++ b/resources/volume/types/volume.go @@ -3,12 +3,12 @@ package types import ( "encoding/json" "fmt" - "reflect" "sort" - "strconv" "strings" "github.com/pkg/errors" + + "github.com/projecteru2/core/utils" ) const auto = "AUTO" @@ -34,7 +34,7 @@ func NewVolumeBinding(volume string) (_ *VolumeBinding, err error) { src, dst, flags = parts[0], parts[1], parts[2] case 4: src, dst, flags = parts[0], parts[1], parts[2] - if size, err = strconv.ParseInt(parts[3], 10, 64); err != nil { + if size, err = utils.ParseRAMInHuman(parts[3]); err != nil { return nil, errors.WithStack(err) } default: @@ -101,6 +101,11 @@ func (vb VolumeBinding) ToString(normalize bool) (volume string) { return volume } +// GetMapKey . +func (vb VolumeBinding) GetMapKey() [3]string { + return [3]string{vb.Source, vb.Destination, vb.Flags} +} + // VolumeBindings is a collection of VolumeBinding type VolumeBindings []*VolumeBinding @@ -116,17 +121,6 @@ func NewVolumeBindings(volumes []string) (volumeBindings VolumeBindings, err err return } -// ToStringSlice converts VolumeBindings into string slice -func (vbs VolumeBindings) ToStringSlice(sorted, normalize bool) (volumes []string) { - if sorted { - sort.Slice(vbs, func(i, j int) bool { return vbs[i].ToString(false) < vbs[j].ToString(false) }) - } - for _, vb := range vbs { - volumes = append(volumes, vb.ToString(normalize)) - } - return -} - // UnmarshalJSON is used for encoding/json.Unmarshal func (vbs *VolumeBindings) UnmarshalJSON(b []byte) (err error) { volumes := []string{} @@ -147,39 +141,33 @@ func (vbs VolumeBindings) MarshalJSON() ([]byte, error) { return bs, errors.WithStack(err) } -// ApplyPlan creates new VolumeBindings according to volume plan -func (vbs VolumeBindings) ApplyPlan(plan VolumePlan) (res VolumeBindings) { +func (vbs VolumeBindings) String() string { + volumes := []string{} for _, vb := range vbs { - newVb := &VolumeBinding{vb.Source, vb.Destination, vb.Flags, vb.SizeInBytes} - if vmap, _ := plan.GetVolumeMap(vb); vmap != nil { - newVb.Source = vmap.GetResourceID() - } - res = append(res, newVb) + volumes = append(volumes, vb.ToString(false)) } - return + return strings.Join(volumes, ",") } -// Divide . -func (vbs VolumeBindings) Divide() (soft VolumeBindings, hard VolumeBindings) { +// TotalSize . +func (vbs VolumeBindings) TotalSize() (total int64) { for _, vb := range vbs { - if strings.HasSuffix(vb.Source, auto) { - soft = append(soft, vb) - } else { - hard = append(hard, vb) - } + total += vb.SizeInBytes } return } -// IsEqual return true is two VolumeBindings have the same value -func (vbs VolumeBindings) IsEqual(vbs2 VolumeBindings) bool { - return reflect.DeepEqual(vbs.ToStringSlice(true, false), vbs2.ToStringSlice(true, false)) -} - -// TotalSize . -func (vbs VolumeBindings) TotalSize() (total int64) { +// ApplyPlan creates new VolumeBindings according to volume plan +func (vbs VolumeBindings) ApplyPlan(plan VolumePlan) (res VolumeBindings) { for _, vb := range vbs { - total += vb.SizeInBytes + newVb := &VolumeBinding{vb.Source, vb.Destination, vb.Flags, vb.SizeInBytes} + if vmap, _ := plan.GetVolumeMap(vb); vmap != nil { + newVb.Source = vmap.GetDevice() + if vmap.GetSize() > newVb.SizeInBytes { + newVb.SizeInBytes = vmap.GetSize() + } + } + res = append(res, newVb) } return } @@ -189,8 +177,7 @@ func MergeVolumeBindings(vbs1 VolumeBindings, vbs2 ...VolumeBindings) (vbs Volum sizeMap := map[[3]string]int64{} // {["AUTO", "/data", "rw"]: 100} for _, vbs := range append(vbs2, vbs1) { for _, vb := range vbs { - key := [3]string{vb.Source, vb.Destination, vb.Flags} - sizeMap[key] += vb.SizeInBytes + sizeMap[vb.GetMapKey()] += vb.SizeInBytes } } @@ -207,3 +194,123 @@ func MergeVolumeBindings(vbs1 VolumeBindings, vbs2 ...VolumeBindings) (vbs Volum } return } + +// VolumeMap . +type VolumeMap map[string]int64 + +// DeepCopy . +func (v VolumeMap) DeepCopy() VolumeMap { + res := VolumeMap{} + for key, value := range v { + res[key] = value + } + return res +} + +// Add . +func (v VolumeMap) Add(v1 VolumeMap) { + for key, value := range v1 { + v[key] += value + } +} + +// Sub . +func (v VolumeMap) Sub(v1 VolumeMap) { + for key, value := range v1 { + v[key] -= value + } +} + +// GetDevice returns the first device +func (v VolumeMap) GetDevice() string { + for key := range v { + return key + } + return "" +} + +// GetSize returns the first size +func (v VolumeMap) GetSize() int64 { + for _, size := range v { + return size + } + return 0 +} + +// Total . +func (v VolumeMap) Total() int64 { + res := int64(0) + for _, size := range v { + res += size + } + return res +} + +// VolumePlan is map from volume string to volumeMap: {"AUTO:/data:rw:100": VolumeMap{"/sda1": 100}} +type VolumePlan map[*VolumeBinding]VolumeMap + +// 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 errors.WithStack(err) + } + for volume, vmap := range plan { + vb, err := NewVolumeBinding(volume) + if err != nil { + return errors.WithStack(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 + } + bs, err := json.Marshal(plan) + return bs, errors.WithStack(err) +} + +// String . +func (p VolumePlan) String() string { + bs, err := p.MarshalJSON() + if err != nil { + return "can not marshal volume plan" + } + return string(bs) +} + +// Merge . +func (p VolumePlan) Merge(p2 VolumePlan) { + for vb, vm := range p2 { + if oldVM, oldVB := p.GetVolumeMap(vb); oldVB != nil { + delete(p, oldVB) + vm[vm.GetDevice()] += oldVM.GetSize() + vm = VolumeMap{vm.GetDevice(): vm.GetSize() + oldVM.GetSize()} + vb = &VolumeBinding{ + Source: vb.Source, + Destination: vb.Destination, + Flags: vb.Flags, + SizeInBytes: vb.SizeInBytes + oldVB.SizeInBytes, + } + } + p[vb] = vm + } +} + +// 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 +} diff --git a/resources/volume/volume.go b/resources/volume/volume.go index c4014bb54..90135f4b7 100644 --- a/resources/volume/volume.go +++ b/resources/volume/volume.go @@ -2,206 +2,334 @@ package volume import ( "context" - "sort" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - "github.com/projecteru2/core/types" - - "github.com/pkg/errors" + enginetypes "github.com/projecteru2/core/engine/types" + "github.com/projecteru2/core/resources" + "github.com/projecteru2/core/resources/volume/models" + "github.com/projecteru2/core/resources/volume/types" + coretypes "github.com/projecteru2/core/types" ) -const maxVolumes = 32 - -type volumeRequest struct { - request [maxVolumes]types.VolumeBinding - limit [maxVolumes]types.VolumeBinding - requests int - limits int - Existing types.VolumePlan -} - -// 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 { - v.request[i] = *vb - } - v.requests = len(opts.VolumeRequest) - v.limits = len(opts.VolumeLimit) - v.Existing = opts.VolumeExist - - 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 { - v.limit[i] = *vb - } - return v, v.Validate() -} - -// Type . -func (v volumeRequest) Type() types.ResourceType { - t := types.ResourceVolume - for i := 0; i < v.requests; i++ { - if v.request[i].RequireSchedule() { - t |= types.ResourceScheduledVolume - break - } +// Plugin wrapper of volume +type Plugin struct { + v *models.Volume +} + +// NewPlugin . +func NewPlugin(config coretypes.Config) (*Plugin, error) { + v, err := models.NewVolume(config) + if err != nil { + return nil, err } - return t + return &Plugin{v: v}, nil } -// Validate . -func (v *volumeRequest) Validate() error { - if v.requests == 0 && v.limits > 0 { - v.request = v.limit - v.requests = v.limits +// GetDeployArgs . +func (v *Plugin) GetDeployArgs(ctx context.Context, nodeName string, deployCount int, resourceOpts coretypes.WorkloadResourceOpts) (*resources.GetDeployArgsResponse, error) { + workloadResourceOpts := &types.WorkloadResourceOpts{} + if err := workloadResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err } - if v.requests != v.limits { - return errors.Wrap(types.ErrBadVolume, "different length of request and limit") + engineArgs, resourceArgs, err := v.v.GetDeployArgs(ctx, nodeName, deployCount, workloadResourceOpts) + if err != nil { + return nil, err } - 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 { - v.limit[i].SizeInBytes = req.SizeInBytes - } - } - return nil + + resp := &resources.GetDeployArgsResponse{} + err = resources.ToResp(map[string]interface{}{ + "engine_args": engineArgs, + "resource_args": resourceArgs, + }, resp) + return resp, err } -// MakeScheduler . -func (v volumeRequest) MakeScheduler() resourcetypes.SchedulerV2 { - return func(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo) (plans resourcetypes.ResourcePlans, total int, err error) { - schedulerV1, err := scheduler.GetSchedulerV1() - if err != nil { - return - } +// GetReallocArgs . +func (v *Plugin) GetReallocArgs(ctx context.Context, nodeName string, originResourceArgs coretypes.WorkloadResourceArgs, resourceOpts coretypes.WorkloadResourceOpts) (*resources.GetReallocArgsResponse, error) { + workloadResourceOpts := &types.WorkloadResourceOpts{} + if err := workloadResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err + } + originWorkloadResourceArgs := &types.WorkloadResourceArgs{} + if err := originWorkloadResourceArgs.ParseFromRawParams(coretypes.RawParams(originResourceArgs)); err != nil { + return nil, err + } - 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]) - } + engineArgs, delta, resourceArgs, err := v.v.GetReallocArgs(ctx, nodeName, originWorkloadResourceArgs, workloadResourceOpts) + if err != nil { + return nil, err + } - var volumePlans map[string][]types.VolumePlan - if v.Existing != nil { - scheduleInfos[0], volumePlans, total, err = schedulerV1.ReselectVolumeNodes(ctx, scheduleInfos[0], v.Existing, request) - } else { - scheduleInfos, volumePlans, total, err = schedulerV1.SelectVolumeNodes(ctx, scheduleInfos, request) - } - return ResourcePlans{ - capacity: resourcetypes.GetCapacity(scheduleInfos), - request: request, - limit: limit, - plan: volumePlans, - }, total, err + resp := &resources.GetReallocArgsResponse{} + err = resources.ToResp(map[string]interface{}{ + "engine_args": engineArgs, + "delta": delta, + "resource_args": resourceArgs, + }, resp) + return resp, err +} + +// GetRemapArgs . +func (v *Plugin) GetRemapArgs(ctx context.Context, nodeName string, workloadMap map[string]*coretypes.Workload) (*resources.GetRemapArgsResponse, error) { + workloadResourceArgsMap, err := v.workloadMapToWorkloadResourceArgsMap(workloadMap) + if err != nil { + return nil, err + } + + engineArgs, err := v.v.GetRemapArgs(ctx, nodeName, workloadResourceArgsMap) + if err != nil { + return nil, err } + + resp := &resources.GetRemapArgsResponse{} + err = resources.ToResp(map[string]interface{}{ + "engine_args": engineArgs, + }, resp) + return resp, err } -// Rate . -func (v volumeRequest) Rate(node types.Node) float64 { - var totalRequest int64 - for i := 0; i < v.requests; i++ { - totalRequest += v.request[i].SizeInBytes +// GetNodesDeployCapacity . +func (v *Plugin) GetNodesDeployCapacity(ctx context.Context, nodeNames []string, resourceOpts coretypes.WorkloadResourceOpts) (*resources.GetNodesDeployCapacityResponse, error) { + workloadResourceOpts := &types.WorkloadResourceOpts{} + if err := workloadResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err + } + + nodesDeployCapacity, total, err := v.v.GetNodesDeployCapacity(ctx, nodeNames, workloadResourceOpts) + if err != nil { + return nil, err } - return float64(totalRequest) / float64(node.Volume.Total()) + + resp := &resources.GetNodesDeployCapacityResponse{} + err = resources.ToResp(map[string]interface{}{ + "nodes": nodesDeployCapacity, + "total": total, + }, resp) + return resp, err } -// ResourcePlans . -type ResourcePlans struct { - capacity map[string]int - request types.VolumeBindings - limit types.VolumeBindings - plan map[string][]types.VolumePlan +// GetMostIdleNode . +func (v *Plugin) GetMostIdleNode(ctx context.Context, nodeNames []string) (*resources.GetMostIdleNodeResponse, error) { + nodeName, priority, err := v.v.GetMostIdleNode(ctx, nodeNames) + if err != nil { + return nil, err + } + + resp := &resources.GetMostIdleNodeResponse{} + err = resources.ToResp(map[string]interface{}{ + "node": nodeName, + "priority": priority, + }, resp) + return resp, err } -// Type . -func (rp ResourcePlans) Type() types.ResourceType { - return types.ResourceVolume +// GetNodeResourceInfo . +func (v *Plugin) GetNodeResourceInfo(ctx context.Context, nodeName string, workloads []*coretypes.Workload) (*resources.GetNodeResourceInfoResponse, error) { + return v.getNodeResourceInfo(ctx, nodeName, workloads, false) } -// Capacity . -func (rp ResourcePlans) Capacity() map[string]int { - return rp.capacity +// FixNodeResource . +func (v *Plugin) FixNodeResource(ctx context.Context, nodeName string, workloads []*coretypes.Workload) (*resources.GetNodeResourceInfoResponse, error) { + return v.getNodeResourceInfo(ctx, nodeName, workloads, true) } -// ApplyChangesOnNode . -func (rp ResourcePlans) ApplyChangesOnNode(node *types.Node, indices ...int) { - if len(rp.plan) == 0 { - return +// SetNodeResourceUsage . +func (v *Plugin) SetNodeResourceUsage(ctx context.Context, nodeName string, resourceOpts coretypes.NodeResourceOpts, resourceArgs coretypes.NodeResourceArgs, workloadResourceArgs []coretypes.WorkloadResourceArgs, delta bool, incr bool) (*resources.SetNodeResourceUsageResponse, error) { + var nodeResourceOpts *types.NodeResourceOpts + var nodeResourceArgs *types.NodeResourceArgs + var workloadResourceArgsList []*types.WorkloadResourceArgs + + if resourceOpts != nil { + nodeResourceOpts = &types.NodeResourceOpts{} + if err := nodeResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err + } } - volumeCost := types.VolumeMap{} - for _, idx := range indices { - plans, ok := rp.plan[node.Name] - if !ok { - continue + if resourceArgs != nil { + nodeResourceArgs = &types.NodeResourceArgs{} + if err := nodeResourceArgs.ParseFromRawParams(coretypes.RawParams(resourceArgs)); err != nil { + return nil, err } - volumeCost.Add(plans[idx].IntoVolumeMap()) } - node.Volume.Sub(volumeCost) - node.SetVolumeUsed(volumeCost.Total(), types.IncrUsage) + + if workloadResourceArgs != nil { + workloadResourceArgsList = make([]*types.WorkloadResourceArgs, len(workloadResourceArgs)) + for i, workloadResourceArg := range workloadResourceArgs { + workloadResourceArgsList[i] = &types.WorkloadResourceArgs{} + if err := workloadResourceArgsList[i].ParseFromRawParams(coretypes.RawParams(workloadResourceArg)); err != nil { + return nil, err + } + } + } + + before, after, err := v.v.SetNodeResourceUsage(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, workloadResourceArgsList, delta, incr) + if err != nil { + return nil, err + } + + resp := &resources.SetNodeResourceUsageResponse{} + err = resources.ToResp(map[string]interface{}{ + "before": before, + "after": after, + }, resp) + return resp, err } -// RollbackChangesOnNode . -func (rp ResourcePlans) RollbackChangesOnNode(node *types.Node, indices ...int) { - if len(rp.plan) == 0 { - return +// SetNodeResourceCapacity . +func (v *Plugin) SetNodeResourceCapacity(ctx context.Context, nodeName string, resourceOpts coretypes.NodeResourceOpts, resourceArgs coretypes.NodeResourceArgs, delta bool, incr bool) (*resources.SetNodeResourceCapacityResponse, error) { + var nodeResourceOpts *types.NodeResourceOpts + var nodeResourceArgs *types.NodeResourceArgs + + if resourceOpts != nil { + nodeResourceOpts = &types.NodeResourceOpts{} + if err := nodeResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err + } + if delta { + nodeResourceOpts.Storage += nodeResourceOpts.Volumes.Total() + } + } + if resourceArgs != nil { + nodeResourceArgs = &types.NodeResourceArgs{} + if err := nodeResourceArgs.ParseFromRawParams(coretypes.RawParams(resourceArgs)); err != nil { + return nil, err + } + if delta { + nodeResourceArgs.Storage += nodeResourceArgs.Volumes.Total() + } } - volumeCost := types.VolumeMap{} - for _, idx := range indices { - volumeCost.Add(rp.plan[node.Name][idx].IntoVolumeMap()) + before, after, err := v.v.SetNodeResourceCapacity(ctx, nodeName, nodeResourceOpts, nodeResourceArgs, delta, incr) + if err != nil { + return nil, err } - node.Volume.Add(volumeCost) - node.SetVolumeUsed(volumeCost.Total(), types.DecrUsage) + + resp := &resources.SetNodeResourceCapacityResponse{} + err = resources.ToResp(map[string]interface{}{ + "before": before, + "after": after, + }, resp) + return resp, err } -// Dispense . -func (rp ResourcePlans) Dispense(opts resourcetypes.DispenseOptions, r *types.ResourceMeta) (*types.ResourceMeta, error) { - if rp.capacity[opts.Node.Name] <= opts.Index { - return nil, errors.WithStack(types.ErrInsufficientCap) +// SetNodeResourceInfo . +func (v *Plugin) SetNodeResourceInfo(ctx context.Context, nodeName string, resourceCapacity coretypes.NodeResourceArgs, resourceUsage coretypes.NodeResourceArgs) (*resources.SetNodeResourceInfoResponse, error) { + capacity := &types.NodeResourceArgs{} + if err := capacity.ParseFromRawParams(coretypes.RawParams(resourceCapacity)); err != nil { + return nil, err } - r.VolumeRequest = rp.request - r.VolumeLimit = rp.limit - if len(rp.plan) == 0 { - return r, nil + + usage := &types.NodeResourceArgs{} + if err := usage.ParseFromRawParams(coretypes.RawParams(resourceUsage)); err != nil { + return nil, err } - if p, ok := rp.plan[opts.Node.Name]; !ok || len(p) <= opts.Index { - return nil, errors.WithStack(types.ErrInsufficientVolume) + if err := v.v.SetNodeResourceInfo(ctx, nodeName, capacity, usage); err != nil { + return nil, err + } + return &resources.SetNodeResourceInfoResponse{}, nil +} + +// AddNode . +func (v *Plugin) AddNode(ctx context.Context, nodeName string, resourceOpts coretypes.NodeResourceOpts, nodeInfo *enginetypes.Info) (*resources.AddNodeResponse, error) { + nodeResourceOpts := &types.NodeResourceOpts{} + if err := nodeResourceOpts.ParseFromRawParams(coretypes.RawParams(resourceOpts)); err != nil { + return nil, err } - r.VolumePlanRequest = rp.plan[opts.Node.Name][opts.Index] - // fix plans while limit > request - r.VolumePlanLimit = types.VolumePlan{} - for i := range rp.request { - request, limit := rp.request[i], rp.limit[i] - if !request.RequireSchedule() { - continue + // set default value + if nodeInfo != nil { + if nodeResourceOpts.Storage == 0 { + nodeResourceOpts.Storage = nodeInfo.StorageTotal * 8 / 10 } - if limit.SizeInBytes > request.SizeInBytes { - p := r.VolumePlanRequest[*request] - r.VolumePlanLimit[*limit] = types.VolumeMap{p.GetResourceID(): p.GetRation() + limit.SizeInBytes - request.SizeInBytes} - } else { - r.VolumePlanLimit[*limit] = r.VolumePlanRequest[*request] + } + + nodeResourceInfo, err := v.v.AddNode(ctx, nodeName, nodeResourceOpts) + if err != nil { + return nil, err + } + + resp := &resources.AddNodeResponse{} + err = resources.ToResp(map[string]interface{}{ + "capacity": nodeResourceInfo.Capacity, + "usage": nodeResourceInfo.Usage, + }, resp) + return resp, err +} + +// RemoveNode . +func (v *Plugin) RemoveNode(ctx context.Context, nodeName string) (*resources.RemoveNodeResponse, error) { + if err := v.v.RemoveNode(ctx, nodeName); err != nil { + return nil, err + } + return &resources.RemoveNodeResponse{}, nil +} + +// Name . +func (v *Plugin) Name() string { + return "volume" +} + +func (v *Plugin) workloadMapToWorkloadResourceArgsMap(workloadMap map[string]*coretypes.Workload) (*types.WorkloadResourceArgsMap, error) { + workloadResourceArgsMap := types.WorkloadResourceArgsMap{} + for workloadID, workload := range workloadMap { + workloadResourceArgs := &types.WorkloadResourceArgs{} + if err := workloadResourceArgs.ParseFromRawParams(coretypes.RawParams(workload.ResourceArgs[v.Name()])); err != nil { + return nil, err } + workloadResourceArgsMap[workloadID] = workloadResourceArgs } - // judge if volume changed - // TODO@zc - r.VolumeChanged = false - return r, nil + return &workloadResourceArgsMap, nil } -// GetPlan return volume plans by nodename -func (rp ResourcePlans) GetPlan(nodename string) []types.VolumePlan { - return rp.plan[nodename] +func (v *Plugin) workloadListToWorkloadResourceArgsMap(workloads []*coretypes.Workload) (*types.WorkloadResourceArgsMap, error) { + workloadMap := map[string]*coretypes.Workload{} + for _, workload := range workloads { + workloadMap[workload.ID] = workload + } + + return v.workloadMapToWorkloadResourceArgsMap(workloadMap) +} + +func (v *Plugin) getNodeResourceInfo(ctx context.Context, nodeName string, workloads []*coretypes.Workload, fix bool) (*resources.GetNodeResourceInfoResponse, error) { + workloadResourceArgsMap, err := v.workloadListToWorkloadResourceArgsMap(workloads) + if err != nil { + return nil, err + } + + nodeResourceInfo, diffs, err := v.v.GetNodeResourceInfo(ctx, nodeName, workloadResourceArgsMap, fix) + if err != nil { + return nil, err + } + + resp := &resources.GetNodeResourceInfoResponse{} + err = resources.ToResp(map[string]interface{}{ + "resource_info": nodeResourceInfo, + "diffs": diffs, + }, resp) + return resp, err +} + +// GetMetricsDescription . +func (v *Plugin) GetMetricsDescription(ctx context.Context) (*resources.GetMetricsDescriptionResponse, error) { + resp := &resources.GetMetricsDescriptionResponse{} + err := resources.ToResp(v.v.GetMetricsDescription(), resp) + return resp, err +} + +// ResolveNodeResourceInfoToMetrics . +func (v *Plugin) ResolveNodeResourceInfoToMetrics(ctx context.Context, podName string, nodeName string, info *resources.NodeResourceInfo) (*resources.ResolveNodeResourceInfoToMetricsResponse, error) { + capacity, usage := &types.NodeResourceArgs{}, &types.NodeResourceArgs{} + if err := capacity.ParseFromRawParams(coretypes.RawParams(info.Capacity)); err != nil { + return nil, err + } + if err := usage.ParseFromRawParams(coretypes.RawParams(info.Usage)); err != nil { + return nil, err + } + + metrics := v.v.ResolveNodeResourceInfoToMetrics(podName, nodeName, capacity, usage) + resp := &resources.ResolveNodeResourceInfoToMetricsResponse{} + err := resources.ToResp(metrics, resp) + return resp, err } diff --git a/resources/volume/volume_test.go b/resources/volume/volume_test.go deleted file mode 100644 index b59c40397..000000000 --- a/resources/volume/volume_test.go +++ /dev/null @@ -1,444 +0,0 @@ -package volume - -import ( - "context" - "testing" - - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/scheduler" - schedulerMocks "github.com/projecteru2/core/scheduler/mocks" - "github.com/projecteru2/core/types" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" -) - -func TestMakeRequest(t *testing.T) { - _, err := MakeRequest(types.ResourceOptions{ - VolumeLimit: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - }, - }, - }) - assert.Nil(t, err) - - // Source not match - _, err = MakeRequest(types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "/data2", - Destination: "/data1", - }, - }, - }) - assert.NotNil(t, err) - - // Dest not match - _, err = MakeRequest(types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data2", - }, - }, - }) - assert.NotNil(t, err) - - // Flag not match - _, err = MakeRequest(types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - Flags: "r", - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - Flags: "rw", - }, - }, - }) - assert.NotNil(t, err) - - // Request SizeInBytes larger then limit - _, err = MakeRequest(types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - SizeInBytes: 10240, - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - SizeInBytes: 5120, - }, - }, - }) - assert.NoError(t, err) - - _, err = MakeRequest(types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - }, - { - Source: "/data2", - Destination: "/data2", - }, - }, - }) - assert.NotNil(t, err) - - _, err = MakeRequest(types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - }, - { - Source: "/data3", - Destination: "/data3", - }, - { - Source: "/data2", - Destination: "/data2", - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - }, - { - Source: "/data4", - Destination: "/data4", - }, - }, - }) - assert.NotNil(t, err) - -} - -func TestType(t *testing.T) { - resourceRequest, err := MakeRequest(types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "AUTO", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "AUTO", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }, - }, - }) - assert.Nil(t, err) - assert.True(t, resourceRequest.Type()&(types.ResourceVolume|types.ResourceScheduledVolume) > 0) -} - -func TestStoragePlans(t *testing.T) { - testStoragePlans(t, types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "AUTO", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "AUTO", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 256, - }, - }, - }) - testStoragePlans(t, types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "AUTO", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "AUTO", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }, - }, - }) -} - -func testStoragePlans(t *testing.T, reqOpts types.ResourceOptions) { - mockScheduler := &schedulerMocks.Scheduler{} - var ( - volumePlans = []types.VolumePlan{ - { - types.VolumeBinding{ - Source: "AUTO", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }: types.VolumeMap{ - "/dev0": 512, - }, - }, - { - types.VolumeBinding{ - Source: "AUTO", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }: types.VolumeMap{ - "/dev1": 512, - }, - }, - } - scheduleInfos []resourcetypes.ScheduleInfo = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "TestNode", - CPU: map[string]int64{"0": 10000, "1": 10000}, - NUMA: map[string]string{"0": "0", "1": "1"}, - NUMAMemory: map[string]int64{"0": 1024, "1": 1204}, - MemCap: 10240, - StorageCap: 10240, - Volume: types.VolumeMap{ - "/data1": 1024, - "/data2": 1024, - }, - InitVolume: types.VolumeMap{ - "/data0": 1024, - }, - }, - VolumePlans: volumePlans, - CPUPlan: []types.CPUMap{{"0": 10000, "1": 10000}}, - Capacity: 100, - }, - } - volumePlan = map[string][]types.VolumePlan{ - "TestNode": volumePlans, - } - ) - - resourceRequest, err := MakeRequest(reqOpts) - assert.Nil(t, err) - assert.True(t, resourceRequest.Type()&types.ResourceVolume > 0) - sche := resourceRequest.MakeScheduler() - - mockScheduler.On( - "SelectVolumeNodes", mock.Anything, mock.Anything, mock.Anything, - ).Return(scheduleInfos, volumePlan, 1, nil) - - prevSche, _ := scheduler.GetSchedulerV1() - scheduler.InitSchedulerV1(nil) - - plans, _, err := sche(context.TODO(), scheduleInfos) - assert.Error(t, err) - - scheduler.InitSchedulerV1(mockScheduler) - defer func() { - scheduler.InitSchedulerV1(prevSche) - }() - - plans, _, err = sche(context.TODO(), scheduleInfos) - assert.Nil(t, err) - assert.True(t, plans.Type()&types.ResourceVolume > 0) - - const storage = int64(10240) - var node = types.Node{ - NodeMeta: types.NodeMeta{ - Name: "TestNode", - CPU: map[string]int64{"0": 10000, "1": 10000}, - NUMA: map[string]string{"0": "0", "1": "1"}, - NUMAMemory: map[string]int64{"0": 1024, "1": 1204}, - MemCap: 10240, - StorageCap: storage, - Volume: types.VolumeMap{"/dev0": 10240, "/dev1": 5120}, - }, - } - - assert.NotNil(t, plans.Capacity()) - plans.ApplyChangesOnNode(&node, 0, 1) - assert.Less(t, node.Volume["/dev0"], int64(10240)) - assert.Less(t, node.Volume["/dev1"], int64(5120)) - - plans.RollbackChangesOnNode(&node, 0, 1) - assert.Equal(t, node.Volume["/dev0"], int64(10240)) - assert.Equal(t, node.Volume["/dev1"], int64(5120)) - - opts := resourcetypes.DispenseOptions{ - Node: &node, - Index: 0, - } - r := &types.ResourceMeta{} - _, err = plans.Dispense(opts, r) - assert.Nil(t, err) - - assert.Nil(t, plans.(ResourcePlans).GetPlan("")) - - opts.Name = "not_exist" - _, err = plans.Dispense(opts, r) - assert.EqualError(t, err, "cannot alloc a each node plan, not enough capacity") - - if reqOpts.VolumeRequest[0].SizeInBytes != reqOpts.VolumeLimit[0].SizeInBytes { - diff := reqOpts.VolumeLimit[0].SizeInBytes - reqOpts.VolumeRequest[0].SizeInBytes - assert.Equal(t, int64(512)+diff, r.VolumePlanLimit[*reqOpts.VolumeLimit[0]]["/dev0"]) - return - } - assert.Equal(t, int64(512), r.VolumePlanLimit[*reqOpts.VolumeRequest[0]]["/dev0"]) -} - -func TestStorage(t *testing.T) { - mockScheduler := &schedulerMocks.Scheduler{} - var ( - volumePlans = []types.VolumePlan{ - { - types.VolumeBinding{ - Source: "/data1", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }: types.VolumeMap{ - "/dev0": 512, - }, - }, - } - scheduleInfos []resourcetypes.ScheduleInfo = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "TestNode", - }, - Capacity: 100, - }, - } - volumePlan = map[string][]types.VolumePlan{ - "TestNode": volumePlans, - } - ) - - resourceRequest, err := MakeRequest(types.ResourceOptions{ - VolumeRequest: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }, - }, - VolumeLimit: []*types.VolumeBinding{ - { - Source: "/data1", - Destination: "/data1", - Flags: "rw", - SizeInBytes: 128, - }, - }, - }) - assert.Nil(t, err) - assert.True(t, resourceRequest.Type()&types.ResourceVolume > 0) - sche := resourceRequest.MakeScheduler() - - mockScheduler.On( - "SelectVolumeNodes", mock.Anything, mock.Anything, mock.Anything, - ).Return(scheduleInfos, volumePlan, 1, nil) - - prevSche, _ := scheduler.GetSchedulerV1() - scheduler.InitSchedulerV1(nil) - - plans, _, err := sche(context.TODO(), scheduleInfos) - assert.Error(t, err) - - scheduler.InitSchedulerV1(mockScheduler) - defer func() { - scheduler.InitSchedulerV1(prevSche) - }() - - plans, _, err = sche(context.TODO(), scheduleInfos) - assert.Nil(t, err) - assert.True(t, plans.Type()&types.ResourceVolume > 0) - - const storage = int64(10240) - var node = types.Node{ - NodeMeta: types.NodeMeta{ - Name: "TestNode", - CPU: map[string]int64{"0": 10000, "1": 10000}, - NUMA: map[string]string{"0": "0", "1": "1"}, - NUMAMemory: map[string]int64{"0": 1024, "1": 1204}, - MemCap: 10240, - StorageCap: storage, - Volume: types.VolumeMap{"/dev0": 10240, "/dev1": 5120}, - }, - VolumeUsed: 0, - } - - assert.NotNil(t, plans.Capacity()) - plans.ApplyChangesOnNode(&node, 0) - assert.Less(t, node.Volume["/dev0"], int64(10240)) - assert.Equal(t, node.Volume["/dev1"], int64(5120)) - - plans.RollbackChangesOnNode(&node, 0) - assert.Equal(t, node.Volume["/dev0"], int64(10240)) - assert.Equal(t, node.Volume["/dev1"], int64(5120)) - - opts := resourcetypes.DispenseOptions{ - Node: &node, - Index: 0, - } - r := &types.ResourceMeta{} - _, err = plans.Dispense(opts, r) - assert.Nil(t, err) -} - -func TestRate(t *testing.T) { - req, err := MakeRequest(types.ResourceOptions{ - VolumeRequest: types.VolumeBindings{&types.VolumeBinding{SizeInBytes: 1024}}, - VolumeLimit: types.VolumeBindings{&types.VolumeBinding{SizeInBytes: 1024}}, - }) - assert.Nil(t, err) - node := types.Node{ - NodeMeta: types.NodeMeta{ - Volume: types.VolumeMap{"1": 1024}, - }, - } - assert.Equal(t, req.Rate(node), 1.0) -} diff --git a/rpc/gen/core.pb.go b/rpc/gen/core.pb.go index 82436bc10..f17108847 100644 --- a/rpc/gen/core.pb.go +++ b/rpc/gen/core.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 -// protoc v3.6.1 +// protoc-gen-go v1.26.0 +// protoc v3.17.3 // source: rpc/gen/core.proto package pb @@ -167,7 +167,7 @@ func (x BuildImageOptions_BuildMethod) Number() protoreflect.EnumNumber { // Deprecated: Use BuildImageOptions_BuildMethod.Descriptor instead. func (BuildImageOptions_BuildMethod) EnumDescriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{44, 0} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{46, 0} } type DeployOptions_Strategy int32 @@ -222,7 +222,7 @@ func (x DeployOptions_Strategy) Number() protoreflect.EnumNumber { // Deprecated: Use DeployOptions_Strategy.Descriptor instead. func (DeployOptions_Strategy) EnumDescriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{52, 0} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{54, 0} } type Empty struct { @@ -689,23 +689,152 @@ func (x *PodResource) GetNodesResource() []*NodeResource { return nil } +type StringSlice struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Slice []string `protobuf:"bytes,1,rep,name=slice,proto3" json:"slice,omitempty"` +} + +func (x *StringSlice) Reset() { + *x = StringSlice{} + if protoimpl.UnsafeEnabled { + mi := &file_rpc_gen_core_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *StringSlice) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StringSlice) ProtoMessage() {} + +func (x *StringSlice) ProtoReflect() protoreflect.Message { + mi := &file_rpc_gen_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) +} + +// Deprecated: Use StringSlice.ProtoReflect.Descriptor instead. +func (*StringSlice) Descriptor() ([]byte, []int) { + return file_rpc_gen_core_proto_rawDescGZIP(), []int{8} +} + +func (x *StringSlice) GetSlice() []string { + if x != nil { + return x.Slice + } + return nil +} + +type RawParam struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Value: + // *RawParam_Str + // *RawParam_StringSlice + Value isRawParam_Value `protobuf_oneof:"value"` +} + +func (x *RawParam) Reset() { + *x = RawParam{} + if protoimpl.UnsafeEnabled { + mi := &file_rpc_gen_core_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RawParam) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RawParam) ProtoMessage() {} + +func (x *RawParam) ProtoReflect() protoreflect.Message { + mi := &file_rpc_gen_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) +} + +// Deprecated: Use RawParam.ProtoReflect.Descriptor instead. +func (*RawParam) Descriptor() ([]byte, []int) { + return file_rpc_gen_core_proto_rawDescGZIP(), []int{9} +} + +func (m *RawParam) GetValue() isRawParam_Value { + if m != nil { + return m.Value + } + return nil +} + +func (x *RawParam) GetStr() string { + if x, ok := x.GetValue().(*RawParam_Str); ok { + return x.Str + } + return "" +} + +func (x *RawParam) GetStringSlice() *StringSlice { + if x, ok := x.GetValue().(*RawParam_StringSlice); ok { + return x.StringSlice + } + return nil +} + +type isRawParam_Value interface { + isRawParam_Value() +} + +type RawParam_Str struct { + Str string `protobuf:"bytes,1,opt,name=str,proto3,oneof"` +} + +type RawParam_StringSlice struct { + StringSlice *StringSlice `protobuf:"bytes,2,opt,name=string_slice,json=stringSlice,proto3,oneof"` +} + +func (*RawParam_Str) isRawParam_Value() {} + +func (*RawParam_StringSlice) isRawParam_Value() {} + type NodeResource struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - 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"` - StoragePercent float64 `protobuf:"fixed64,4,opt,name=storage_percent,json=storagePercent,proto3" json:"storage_percent,omitempty"` - VolumePercent float64 `protobuf:"fixed64,5,opt,name=volume_percent,json=volumePercent,proto3" json:"volume_percent,omitempty"` - Diffs []string `protobuf:"bytes,6,rep,name=diffs,proto3" json:"diffs,omitempty"` + 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"` + StoragePercent float64 `protobuf:"fixed64,4,opt,name=storage_percent,json=storagePercent,proto3" json:"storage_percent,omitempty"` + VolumePercent float64 `protobuf:"fixed64,5,opt,name=volume_percent,json=volumePercent,proto3" json:"volume_percent,omitempty"` + Diffs []string `protobuf:"bytes,6,rep,name=diffs,proto3" json:"diffs,omitempty"` + ResourceCapacity string `protobuf:"bytes,7,opt,name=resource_capacity,json=resourceCapacity,proto3" json:"resource_capacity,omitempty"` + ResourceUsage string `protobuf:"bytes,8,opt,name=resource_usage,json=resourceUsage,proto3" json:"resource_usage,omitempty"` } func (x *NodeResource) Reset() { *x = NodeResource{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[8] + mi := &file_rpc_gen_core_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -718,7 +847,7 @@ func (x *NodeResource) String() string { func (*NodeResource) ProtoMessage() {} func (x *NodeResource) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[8] + mi := &file_rpc_gen_core_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -731,7 +860,7 @@ func (x *NodeResource) ProtoReflect() protoreflect.Message { // Deprecated: Use NodeResource.ProtoReflect.Descriptor instead. func (*NodeResource) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{8} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{10} } func (x *NodeResource) GetName() string { @@ -776,6 +905,20 @@ func (x *NodeResource) GetDiffs() []string { return nil } +func (x *NodeResource) GetResourceCapacity() string { + if x != nil { + return x.ResourceCapacity + } + return "" +} + +func (x *NodeResource) GetResourceUsage() string { + if x != nil { + return x.ResourceUsage + } + return "" +} + type ListNetworkOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -788,7 +931,7 @@ type ListNetworkOptions struct { func (x *ListNetworkOptions) Reset() { *x = ListNetworkOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[9] + mi := &file_rpc_gen_core_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -801,7 +944,7 @@ func (x *ListNetworkOptions) String() string { func (*ListNetworkOptions) ProtoMessage() {} func (x *ListNetworkOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[9] + mi := &file_rpc_gen_core_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -814,7 +957,7 @@ func (x *ListNetworkOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use ListNetworkOptions.ProtoReflect.Descriptor instead. func (*ListNetworkOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{9} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{11} } func (x *ListNetworkOptions) GetPodname() string { @@ -845,7 +988,7 @@ type ConnectNetworkOptions struct { func (x *ConnectNetworkOptions) Reset() { *x = ConnectNetworkOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[10] + mi := &file_rpc_gen_core_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -858,7 +1001,7 @@ func (x *ConnectNetworkOptions) String() string { func (*ConnectNetworkOptions) ProtoMessage() {} func (x *ConnectNetworkOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[10] + mi := &file_rpc_gen_core_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -871,7 +1014,7 @@ func (x *ConnectNetworkOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use ConnectNetworkOptions.ProtoReflect.Descriptor instead. func (*ConnectNetworkOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{10} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{12} } func (x *ConnectNetworkOptions) GetNetwork() string { @@ -915,7 +1058,7 @@ type DisconnectNetworkOptions struct { func (x *DisconnectNetworkOptions) Reset() { *x = DisconnectNetworkOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[11] + mi := &file_rpc_gen_core_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -928,7 +1071,7 @@ func (x *DisconnectNetworkOptions) String() string { func (*DisconnectNetworkOptions) ProtoMessage() {} func (x *DisconnectNetworkOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[11] + mi := &file_rpc_gen_core_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -941,7 +1084,7 @@ func (x *DisconnectNetworkOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use DisconnectNetworkOptions.ProtoReflect.Descriptor instead. func (*DisconnectNetworkOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{11} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{13} } func (x *DisconnectNetworkOptions) GetNetwork() string { @@ -977,7 +1120,7 @@ type Network struct { func (x *Network) Reset() { *x = Network{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[12] + mi := &file_rpc_gen_core_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -990,7 +1133,7 @@ func (x *Network) String() string { func (*Network) ProtoMessage() {} func (x *Network) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[12] + mi := &file_rpc_gen_core_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1003,7 +1146,7 @@ func (x *Network) ProtoReflect() protoreflect.Message { // Deprecated: Use Network.ProtoReflect.Descriptor instead. func (*Network) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{12} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{14} } func (x *Network) GetName() string { @@ -1031,7 +1174,7 @@ type Networks struct { func (x *Networks) Reset() { *x = Networks{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[13] + mi := &file_rpc_gen_core_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1044,7 +1187,7 @@ func (x *Networks) String() string { func (*Networks) ProtoMessage() {} func (x *Networks) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[13] + mi := &file_rpc_gen_core_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1057,7 +1200,7 @@ func (x *Networks) ProtoReflect() protoreflect.Message { // Deprecated: Use Networks.ProtoReflect.Descriptor instead. func (*Networks) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{13} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{15} } func (x *Networks) GetNetworks() []*Network { @@ -1072,34 +1215,36 @@ type Node struct { 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"` - InitNumaMemory map[string]int64 `protobuf:"bytes,21,rep,name=init_numa_memory,json=initNumaMemory,proto3" json:"init_numa_memory,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Bypass bool `protobuf:"varint,22,opt,name=bypass,proto3" json:"bypass,omitempty"` + 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"` + InitNumaMemory map[string]int64 `protobuf:"bytes,21,rep,name=init_numa_memory,json=initNumaMemory,proto3" json:"init_numa_memory,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Bypass bool `protobuf:"varint,22,opt,name=bypass,proto3" json:"bypass,omitempty"` + ResourceCapacity string `protobuf:"bytes,23,opt,name=resource_capacity,json=resourceCapacity,proto3" json:"resource_capacity,omitempty"` + ResourceUsage string `protobuf:"bytes,24,opt,name=resource_usage,json=resourceUsage,proto3" json:"resource_usage,omitempty"` } func (x *Node) Reset() { *x = Node{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[14] + mi := &file_rpc_gen_core_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1112,7 +1257,7 @@ func (x *Node) String() string { func (*Node) ProtoMessage() {} func (x *Node) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[14] + mi := &file_rpc_gen_core_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1125,7 +1270,7 @@ func (x *Node) ProtoReflect() protoreflect.Message { // Deprecated: Use Node.ProtoReflect.Descriptor instead. func (*Node) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{14} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{16} } func (x *Node) GetName() string { @@ -1282,6 +1427,20 @@ func (x *Node) GetBypass() bool { return false } +func (x *Node) GetResourceCapacity() string { + if x != nil { + return x.ResourceCapacity + } + return "" +} + +func (x *Node) GetResourceUsage() string { + if x != nil { + return x.ResourceUsage + } + return "" +} + type Nodes struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1293,7 +1452,7 @@ type Nodes struct { func (x *Nodes) Reset() { *x = Nodes{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[15] + mi := &file_rpc_gen_core_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1306,7 +1465,7 @@ func (x *Nodes) String() string { func (*Nodes) ProtoMessage() {} func (x *Nodes) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[15] + mi := &file_rpc_gen_core_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1319,7 +1478,7 @@ func (x *Nodes) ProtoReflect() protoreflect.Message { // Deprecated: Use Nodes.ProtoReflect.Descriptor instead. func (*Nodes) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{15} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{17} } func (x *Nodes) GetNodes() []*Node { @@ -1341,7 +1500,7 @@ type NodeAvailable struct { func (x *NodeAvailable) Reset() { *x = NodeAvailable{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[16] + mi := &file_rpc_gen_core_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1354,7 +1513,7 @@ func (x *NodeAvailable) String() string { func (*NodeAvailable) ProtoMessage() {} func (x *NodeAvailable) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[16] + mi := &file_rpc_gen_core_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1367,7 +1526,7 @@ func (x *NodeAvailable) ProtoReflect() protoreflect.Message { // Deprecated: Use NodeAvailable.ProtoReflect.Descriptor instead. func (*NodeAvailable) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{16} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{18} } func (x *NodeAvailable) GetNodename() string { @@ -1389,26 +1548,28 @@ type SetNodeOptions struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,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"` - WorkloadsDown bool `protobuf:"varint,10,opt,name=workloads_down,json=workloadsDown,proto3" json:"workloads_down,omitempty"` - Endpoint string `protobuf:"bytes,11,opt,name=endpoint,proto3" json:"endpoint,omitempty"` - BypassOpt TriOpt `protobuf:"varint,12,opt,name=bypass_opt,json=bypassOpt,proto3,enum=pb.TriOpt" json:"bypass_opt,omitempty"` - Ca string `protobuf:"bytes,13,opt,name=ca,proto3" json:"ca,omitempty"` - Cert string `protobuf:"bytes,14,opt,name=cert,proto3" json:"cert,omitempty"` - Key string `protobuf:"bytes,15,opt,name=key,proto3" json:"key,omitempty"` + Nodename string `protobuf:"bytes,1,opt,name=nodename,proto3" json:"nodename,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"` + WorkloadsDown bool `protobuf:"varint,10,opt,name=workloads_down,json=workloadsDown,proto3" json:"workloads_down,omitempty"` + Endpoint string `protobuf:"bytes,11,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + BypassOpt TriOpt `protobuf:"varint,12,opt,name=bypass_opt,json=bypassOpt,proto3,enum=pb.TriOpt" json:"bypass_opt,omitempty"` + Ca string `protobuf:"bytes,13,opt,name=ca,proto3" json:"ca,omitempty"` + Cert string `protobuf:"bytes,14,opt,name=cert,proto3" json:"cert,omitempty"` + Key string `protobuf:"bytes,15,opt,name=key,proto3" json:"key,omitempty"` + ResourceOpts map[string]*RawParam `protobuf:"bytes,16,rep,name=resource_opts,json=resourceOpts,proto3" json:"resource_opts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Delta bool `protobuf:"varint,17,opt,name=delta,proto3" json:"delta,omitempty"` } func (x *SetNodeOptions) Reset() { *x = SetNodeOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[17] + mi := &file_rpc_gen_core_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1421,7 +1582,7 @@ func (x *SetNodeOptions) String() string { func (*SetNodeOptions) ProtoMessage() {} func (x *SetNodeOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[17] + mi := &file_rpc_gen_core_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1434,7 +1595,7 @@ func (x *SetNodeOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use SetNodeOptions.ProtoReflect.Descriptor instead. func (*SetNodeOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{17} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{19} } func (x *SetNodeOptions) GetNodename() string { @@ -1535,6 +1696,20 @@ func (x *SetNodeOptions) GetKey() string { return "" } +func (x *SetNodeOptions) GetResourceOpts() map[string]*RawParam { + if x != nil { + return x.ResourceOpts + } + return nil +} + +func (x *SetNodeOptions) GetDelta() bool { + if x != nil { + return x.Delta + } + return false +} + type SetNodeStatusOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1547,7 +1722,7 @@ type SetNodeStatusOptions struct { func (x *SetNodeStatusOptions) Reset() { *x = SetNodeStatusOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[18] + mi := &file_rpc_gen_core_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1560,7 +1735,7 @@ func (x *SetNodeStatusOptions) String() string { func (*SetNodeStatusOptions) ProtoMessage() {} func (x *SetNodeStatusOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[18] + mi := &file_rpc_gen_core_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1573,7 +1748,7 @@ func (x *SetNodeStatusOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use SetNodeStatusOptions.ProtoReflect.Descriptor instead. func (*SetNodeStatusOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{18} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{20} } func (x *SetNodeStatusOptions) GetNodename() string { @@ -1601,7 +1776,7 @@ type GetNodeStatusOptions struct { func (x *GetNodeStatusOptions) Reset() { *x = GetNodeStatusOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[19] + mi := &file_rpc_gen_core_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1614,7 +1789,7 @@ func (x *GetNodeStatusOptions) String() string { func (*GetNodeStatusOptions) ProtoMessage() {} func (x *GetNodeStatusOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[19] + mi := &file_rpc_gen_core_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1627,7 +1802,7 @@ func (x *GetNodeStatusOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use GetNodeStatusOptions.ProtoReflect.Descriptor instead. func (*GetNodeStatusOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{19} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{21} } func (x *GetNodeStatusOptions) GetNodename() string { @@ -1651,7 +1826,7 @@ type NodeStatusStreamMessage struct { func (x *NodeStatusStreamMessage) Reset() { *x = NodeStatusStreamMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[20] + mi := &file_rpc_gen_core_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1664,7 +1839,7 @@ func (x *NodeStatusStreamMessage) String() string { func (*NodeStatusStreamMessage) ProtoMessage() {} func (x *NodeStatusStreamMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[20] + mi := &file_rpc_gen_core_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1677,7 +1852,7 @@ func (x *NodeStatusStreamMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use NodeStatusStreamMessage.ProtoReflect.Descriptor instead. func (*NodeStatusStreamMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{20} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{22} } func (x *NodeStatusStreamMessage) GetNodename() string { @@ -1722,7 +1897,7 @@ type NodeFilter struct { func (x *NodeFilter) Reset() { *x = NodeFilter{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[21] + mi := &file_rpc_gen_core_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1735,7 +1910,7 @@ func (x *NodeFilter) String() string { func (*NodeFilter) ProtoMessage() {} func (x *NodeFilter) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[21] + mi := &file_rpc_gen_core_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1748,7 +1923,7 @@ func (x *NodeFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use NodeFilter.ProtoReflect.Descriptor instead. func (*NodeFilter) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{21} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{23} } func (x *NodeFilter) GetIncludes() []string { @@ -1784,24 +1959,25 @@ type Workload struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - 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 *WorkloadStatus `protobuf:"bytes,9,opt,name=status,proto3" json:"status,omitempty"` - Resource *Resource `protobuf:"bytes,10,opt,name=resource,proto3" json:"resource,omitempty"` - CreateTime int64 `protobuf:"varint,11,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` - Env []string `protobuf:"bytes,12,rep,name=env,proto3" json:"env,omitempty"` + 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 *WorkloadStatus `protobuf:"bytes,9,opt,name=status,proto3" json:"status,omitempty"` + Resource *Resource `protobuf:"bytes,10,opt,name=resource,proto3" json:"resource,omitempty"` + CreateTime int64 `protobuf:"varint,11,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` + Env []string `protobuf:"bytes,12,rep,name=env,proto3" json:"env,omitempty"` + ResourceArgs string `protobuf:"bytes,13,opt,name=resource_args,json=resourceArgs,proto3" json:"resource_args,omitempty"` } func (x *Workload) Reset() { *x = Workload{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[22] + mi := &file_rpc_gen_core_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1814,7 +1990,7 @@ func (x *Workload) String() string { func (*Workload) ProtoMessage() {} func (x *Workload) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[22] + mi := &file_rpc_gen_core_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1827,7 +2003,7 @@ func (x *Workload) ProtoReflect() protoreflect.Message { // Deprecated: Use Workload.ProtoReflect.Descriptor instead. func (*Workload) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{22} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{24} } func (x *Workload) GetId() string { @@ -1914,6 +2090,13 @@ func (x *Workload) GetEnv() []string { return nil } +func (x *Workload) GetResourceArgs() string { + if x != nil { + return x.ResourceArgs + } + return "" +} + type WorkloadStatus struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1925,7 +2108,7 @@ type WorkloadStatus struct { 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"` - // extra fields used to set workload status + // extra fields used to set workload status Appname string `protobuf:"bytes,7,opt,name=appname,proto3" json:"appname,omitempty"` Nodename string `protobuf:"bytes,8,opt,name=nodename,proto3" json:"nodename,omitempty"` Entrypoint string `protobuf:"bytes,9,opt,name=entrypoint,proto3" json:"entrypoint,omitempty"` @@ -1934,7 +2117,7 @@ type WorkloadStatus struct { func (x *WorkloadStatus) Reset() { *x = WorkloadStatus{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[23] + mi := &file_rpc_gen_core_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1947,7 +2130,7 @@ func (x *WorkloadStatus) String() string { func (*WorkloadStatus) ProtoMessage() {} func (x *WorkloadStatus) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[23] + mi := &file_rpc_gen_core_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1960,7 +2143,7 @@ func (x *WorkloadStatus) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkloadStatus.ProtoReflect.Descriptor instead. func (*WorkloadStatus) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{23} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{25} } func (x *WorkloadStatus) GetId() string { @@ -2037,7 +2220,7 @@ type WorkloadsStatus struct { func (x *WorkloadsStatus) Reset() { *x = WorkloadsStatus{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[24] + mi := &file_rpc_gen_core_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2050,7 +2233,7 @@ func (x *WorkloadsStatus) String() string { func (*WorkloadsStatus) ProtoMessage() {} func (x *WorkloadsStatus) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[24] + mi := &file_rpc_gen_core_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2063,7 +2246,7 @@ func (x *WorkloadsStatus) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkloadsStatus.ProtoReflect.Descriptor instead. func (*WorkloadsStatus) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{24} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{26} } func (x *WorkloadsStatus) GetStatus() []*WorkloadStatus { @@ -2084,7 +2267,7 @@ type SetWorkloadsStatusOptions struct { func (x *SetWorkloadsStatusOptions) Reset() { *x = SetWorkloadsStatusOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[25] + mi := &file_rpc_gen_core_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2097,7 +2280,7 @@ func (x *SetWorkloadsStatusOptions) String() string { func (*SetWorkloadsStatusOptions) ProtoMessage() {} func (x *SetWorkloadsStatusOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[25] + mi := &file_rpc_gen_core_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2110,7 +2293,7 @@ func (x *SetWorkloadsStatusOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use SetWorkloadsStatusOptions.ProtoReflect.Descriptor instead. func (*SetWorkloadsStatusOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{25} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{27} } func (x *SetWorkloadsStatusOptions) GetStatus() []*WorkloadStatus { @@ -2134,7 +2317,7 @@ type WorkloadStatusStreamOptions struct { func (x *WorkloadStatusStreamOptions) Reset() { *x = WorkloadStatusStreamOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[26] + mi := &file_rpc_gen_core_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2147,7 +2330,7 @@ func (x *WorkloadStatusStreamOptions) String() string { func (*WorkloadStatusStreamOptions) ProtoMessage() {} func (x *WorkloadStatusStreamOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[26] + mi := &file_rpc_gen_core_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2160,7 +2343,7 @@ func (x *WorkloadStatusStreamOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkloadStatusStreamOptions.ProtoReflect.Descriptor instead. func (*WorkloadStatusStreamOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{26} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{28} } func (x *WorkloadStatusStreamOptions) GetAppname() string { @@ -2206,7 +2389,7 @@ type WorkloadStatusStreamMessage struct { func (x *WorkloadStatusStreamMessage) Reset() { *x = WorkloadStatusStreamMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[27] + mi := &file_rpc_gen_core_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2219,7 +2402,7 @@ func (x *WorkloadStatusStreamMessage) String() string { func (*WorkloadStatusStreamMessage) ProtoMessage() {} func (x *WorkloadStatusStreamMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[27] + mi := &file_rpc_gen_core_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2232,7 +2415,7 @@ func (x *WorkloadStatusStreamMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkloadStatusStreamMessage.ProtoReflect.Descriptor instead. func (*WorkloadStatusStreamMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{27} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{29} } func (x *WorkloadStatusStreamMessage) GetId() string { @@ -2281,7 +2464,7 @@ type Workloads struct { func (x *Workloads) Reset() { *x = Workloads{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[28] + mi := &file_rpc_gen_core_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2294,7 +2477,7 @@ func (x *Workloads) String() string { func (*Workloads) ProtoMessage() {} func (x *Workloads) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[28] + mi := &file_rpc_gen_core_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2307,7 +2490,7 @@ func (x *Workloads) ProtoReflect() protoreflect.Message { // Deprecated: Use Workloads.ProtoReflect.Descriptor instead. func (*Workloads) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{28} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{30} } func (x *Workloads) GetWorkloads() []*Workload { @@ -2328,7 +2511,7 @@ type WorkloadID struct { func (x *WorkloadID) Reset() { *x = WorkloadID{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[29] + mi := &file_rpc_gen_core_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2341,7 +2524,7 @@ func (x *WorkloadID) String() string { func (*WorkloadID) ProtoMessage() {} func (x *WorkloadID) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[29] + mi := &file_rpc_gen_core_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2354,7 +2537,7 @@ func (x *WorkloadID) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkloadID.ProtoReflect.Descriptor instead. func (*WorkloadID) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{29} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{31} } func (x *WorkloadID) GetId() string { @@ -2375,7 +2558,7 @@ type WorkloadIDs struct { func (x *WorkloadIDs) Reset() { *x = WorkloadIDs{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[30] + mi := &file_rpc_gen_core_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2388,7 +2571,7 @@ func (x *WorkloadIDs) String() string { func (*WorkloadIDs) ProtoMessage() {} func (x *WorkloadIDs) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[30] + mi := &file_rpc_gen_core_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2401,7 +2584,7 @@ func (x *WorkloadIDs) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkloadIDs.ProtoReflect.Descriptor instead. func (*WorkloadIDs) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{30} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{32} } func (x *WorkloadIDs) GetIds() []string { @@ -2418,12 +2601,13 @@ type RemoveWorkloadOptions 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"` } func (x *RemoveWorkloadOptions) Reset() { *x = RemoveWorkloadOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[31] + mi := &file_rpc_gen_core_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2436,7 +2620,7 @@ func (x *RemoveWorkloadOptions) String() string { func (*RemoveWorkloadOptions) ProtoMessage() {} func (x *RemoveWorkloadOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[31] + mi := &file_rpc_gen_core_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2449,7 +2633,7 @@ func (x *RemoveWorkloadOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveWorkloadOptions.ProtoReflect.Descriptor instead. func (*RemoveWorkloadOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{31} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{33} } func (x *RemoveWorkloadOptions) GetIds() []string { @@ -2466,6 +2650,13 @@ func (x *RemoveWorkloadOptions) GetForce() bool { return false } +func (x *RemoveWorkloadOptions) GetStep() int32 { + if x != nil { + return x.Step + } + return 0 +} + type DissociateWorkloadOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2477,7 +2668,7 @@ type DissociateWorkloadOptions struct { func (x *DissociateWorkloadOptions) Reset() { *x = DissociateWorkloadOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[32] + mi := &file_rpc_gen_core_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2490,7 +2681,7 @@ func (x *DissociateWorkloadOptions) String() string { func (*DissociateWorkloadOptions) ProtoMessage() {} func (x *DissociateWorkloadOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[32] + mi := &file_rpc_gen_core_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2503,7 +2694,7 @@ func (x *DissociateWorkloadOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use DissociateWorkloadOptions.ProtoReflect.Descriptor instead. func (*DissociateWorkloadOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{32} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{34} } func (x *DissociateWorkloadOptions) GetIds() []string { @@ -2518,15 +2709,16 @@ type ReallocOptions struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - 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"` + 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"` // deprecated + OldResourceOpts *ResourceOptions `protobuf:"bytes,3,opt,name=old_resource_opts,json=oldResourceOpts,proto3" json:"old_resource_opts,omitempty"` + ResourceOpts map[string]*RawParam `protobuf:"bytes,4,rep,name=resource_opts,json=resourceOpts,proto3" json:"resource_opts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *ReallocOptions) Reset() { *x = ReallocOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[33] + mi := &file_rpc_gen_core_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2539,7 +2731,7 @@ func (x *ReallocOptions) String() string { func (*ReallocOptions) ProtoMessage() {} func (x *ReallocOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[33] + mi := &file_rpc_gen_core_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2552,7 +2744,7 @@ func (x *ReallocOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use ReallocOptions.ProtoReflect.Descriptor instead. func (*ReallocOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{33} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{35} } func (x *ReallocOptions) GetId() string { @@ -2569,7 +2761,14 @@ func (x *ReallocOptions) GetBindCpuOpt() TriOpt { return TriOpt_KEEP } -func (x *ReallocOptions) GetResourceOpts() *ResourceOptions { +func (x *ReallocOptions) GetOldResourceOpts() *ResourceOptions { + if x != nil { + return x.OldResourceOpts + } + return nil +} + +func (x *ReallocOptions) GetResourceOpts() map[string]*RawParam { if x != nil { return x.ResourceOpts } @@ -2588,7 +2787,7 @@ type AddPodOptions struct { func (x *AddPodOptions) Reset() { *x = AddPodOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[34] + mi := &file_rpc_gen_core_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2601,7 +2800,7 @@ func (x *AddPodOptions) String() string { func (*AddPodOptions) ProtoMessage() {} func (x *AddPodOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[34] + mi := &file_rpc_gen_core_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2614,7 +2813,7 @@ func (x *AddPodOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use AddPodOptions.ProtoReflect.Descriptor instead. func (*AddPodOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{34} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{36} } func (x *AddPodOptions) GetName() string { @@ -2642,7 +2841,7 @@ type RemovePodOptions struct { func (x *RemovePodOptions) Reset() { *x = RemovePodOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[35] + mi := &file_rpc_gen_core_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2655,7 +2854,7 @@ func (x *RemovePodOptions) String() string { func (*RemovePodOptions) ProtoMessage() {} func (x *RemovePodOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[35] + mi := &file_rpc_gen_core_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2668,7 +2867,7 @@ func (x *RemovePodOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use RemovePodOptions.ProtoReflect.Descriptor instead. func (*RemovePodOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{35} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{37} } func (x *RemovePodOptions) GetName() string { @@ -2689,7 +2888,7 @@ type GetPodOptions struct { func (x *GetPodOptions) Reset() { *x = GetPodOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[36] + mi := &file_rpc_gen_core_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2702,7 +2901,7 @@ func (x *GetPodOptions) String() string { func (*GetPodOptions) ProtoMessage() {} func (x *GetPodOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[36] + mi := &file_rpc_gen_core_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2715,7 +2914,7 @@ func (x *GetPodOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use GetPodOptions.ProtoReflect.Descriptor instead. func (*GetPodOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{36} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{38} } func (x *GetPodOptions) GetName() string { @@ -2730,26 +2929,27 @@ type AddNodeOptions struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - 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"` + 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"` + ResourceOpts map[string]*RawParam `protobuf:"bytes,15,rep,name=resource_opts,json=resourceOpts,proto3" json:"resource_opts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *AddNodeOptions) Reset() { *x = AddNodeOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[37] + mi := &file_rpc_gen_core_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2762,7 +2962,7 @@ func (x *AddNodeOptions) String() string { func (*AddNodeOptions) ProtoMessage() {} func (x *AddNodeOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[37] + mi := &file_rpc_gen_core_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2775,7 +2975,7 @@ func (x *AddNodeOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use AddNodeOptions.ProtoReflect.Descriptor instead. func (*AddNodeOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{37} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{39} } func (x *AddNodeOptions) GetNodename() string { @@ -2876,6 +3076,13 @@ func (x *AddNodeOptions) GetVolumeMap() map[string]int64 { return nil } +func (x *AddNodeOptions) GetResourceOpts() map[string]*RawParam { + if x != nil { + return x.ResourceOpts + } + return nil +} + type RemoveNodeOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2887,7 +3094,7 @@ type RemoveNodeOptions struct { func (x *RemoveNodeOptions) Reset() { *x = RemoveNodeOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[38] + mi := &file_rpc_gen_core_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2900,7 +3107,7 @@ func (x *RemoveNodeOptions) String() string { func (*RemoveNodeOptions) ProtoMessage() {} func (x *RemoveNodeOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[38] + mi := &file_rpc_gen_core_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2913,7 +3120,7 @@ func (x *RemoveNodeOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveNodeOptions.ProtoReflect.Descriptor instead. func (*RemoveNodeOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{38} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{40} } func (x *RemoveNodeOptions) GetNodename() string { @@ -2935,7 +3142,7 @@ type GetNodeOptions struct { func (x *GetNodeOptions) Reset() { *x = GetNodeOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[39] + mi := &file_rpc_gen_core_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2948,7 +3155,7 @@ func (x *GetNodeOptions) String() string { func (*GetNodeOptions) ProtoMessage() {} func (x *GetNodeOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[39] + mi := &file_rpc_gen_core_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2961,7 +3168,7 @@ func (x *GetNodeOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use GetNodeOptions.ProtoReflect.Descriptor instead. func (*GetNodeOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{39} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{41} } func (x *GetNodeOptions) GetNodename() string { @@ -2990,7 +3197,7 @@ type GetNodeResourceOptions struct { func (x *GetNodeResourceOptions) Reset() { *x = GetNodeResourceOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[40] + mi := &file_rpc_gen_core_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3003,7 +3210,7 @@ func (x *GetNodeResourceOptions) String() string { func (*GetNodeResourceOptions) ProtoMessage() {} func (x *GetNodeResourceOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[40] + mi := &file_rpc_gen_core_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3016,7 +3223,7 @@ func (x *GetNodeResourceOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use GetNodeResourceOptions.ProtoReflect.Descriptor instead. func (*GetNodeResourceOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{40} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{42} } func (x *GetNodeResourceOptions) GetOpts() *GetNodeOptions { @@ -3048,7 +3255,7 @@ type ListNodesOptions struct { func (x *ListNodesOptions) Reset() { *x = ListNodesOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[41] + mi := &file_rpc_gen_core_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3061,7 +3268,7 @@ func (x *ListNodesOptions) String() string { func (*ListNodesOptions) ProtoMessage() {} func (x *ListNodesOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[41] + mi := &file_rpc_gen_core_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3074,7 +3281,7 @@ func (x *ListNodesOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use ListNodesOptions.ProtoReflect.Descriptor instead. func (*ListNodesOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{41} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{43} } func (x *ListNodesOptions) GetPodname() string { @@ -3135,7 +3342,7 @@ type Build struct { func (x *Build) Reset() { *x = Build{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[42] + mi := &file_rpc_gen_core_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3148,7 +3355,7 @@ func (x *Build) String() string { func (*Build) ProtoMessage() {} func (x *Build) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[42] + mi := &file_rpc_gen_core_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3161,7 +3368,7 @@ func (x *Build) ProtoReflect() protoreflect.Message { // Deprecated: Use Build.ProtoReflect.Descriptor instead. func (*Build) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{42} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{44} } func (x *Build) GetBase() string { @@ -3267,7 +3474,7 @@ type Builds struct { func (x *Builds) Reset() { *x = Builds{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[43] + mi := &file_rpc_gen_core_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3280,7 +3487,7 @@ func (x *Builds) String() string { func (*Builds) ProtoMessage() {} func (x *Builds) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[43] + mi := &file_rpc_gen_core_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3293,7 +3500,7 @@ func (x *Builds) ProtoReflect() protoreflect.Message { // Deprecated: Use Builds.ProtoReflect.Descriptor instead. func (*Builds) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{43} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{45} } func (x *Builds) GetStages() []string { @@ -3328,7 +3535,7 @@ type BuildImageOptions struct { func (x *BuildImageOptions) Reset() { *x = BuildImageOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[44] + mi := &file_rpc_gen_core_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3341,7 +3548,7 @@ func (x *BuildImageOptions) String() string { func (*BuildImageOptions) ProtoMessage() {} func (x *BuildImageOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[44] + mi := &file_rpc_gen_core_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3354,7 +3561,7 @@ func (x *BuildImageOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use BuildImageOptions.ProtoReflect.Descriptor instead. func (*BuildImageOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{44} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{46} } func (x *BuildImageOptions) GetName() string { @@ -3426,7 +3633,7 @@ type HookOptions struct { func (x *HookOptions) Reset() { *x = HookOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[45] + mi := &file_rpc_gen_core_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3439,7 +3646,7 @@ func (x *HookOptions) String() string { func (*HookOptions) ProtoMessage() {} func (x *HookOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[45] + mi := &file_rpc_gen_core_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3452,7 +3659,7 @@ func (x *HookOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use HookOptions.ProtoReflect.Descriptor instead. func (*HookOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{45} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{47} } func (x *HookOptions) GetAfterStart() []string { @@ -3490,7 +3697,7 @@ type HealthCheckOptions struct { func (x *HealthCheckOptions) Reset() { *x = HealthCheckOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[46] + mi := &file_rpc_gen_core_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3503,7 +3710,7 @@ func (x *HealthCheckOptions) String() string { func (*HealthCheckOptions) ProtoMessage() {} func (x *HealthCheckOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[46] + mi := &file_rpc_gen_core_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3516,7 +3723,7 @@ func (x *HealthCheckOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use HealthCheckOptions.ProtoReflect.Descriptor instead. func (*HealthCheckOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{46} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{48} } func (x *HealthCheckOptions) GetTcpPorts() []string { @@ -3559,7 +3766,7 @@ type LogOptions struct { func (x *LogOptions) Reset() { *x = LogOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[47] + mi := &file_rpc_gen_core_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3572,7 +3779,7 @@ func (x *LogOptions) String() string { func (*LogOptions) ProtoMessage() {} func (x *LogOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[47] + mi := &file_rpc_gen_core_proto_msgTypes[49] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3585,7 +3792,7 @@ func (x *LogOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use LogOptions.ProtoReflect.Descriptor instead. func (*LogOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{47} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{49} } func (x *LogOptions) GetType() string { @@ -3627,7 +3834,7 @@ type EntrypointOptions struct { func (x *EntrypointOptions) Reset() { *x = EntrypointOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[48] + mi := &file_rpc_gen_core_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3640,7 +3847,7 @@ func (x *EntrypointOptions) String() string { func (*EntrypointOptions) ProtoMessage() {} func (x *EntrypointOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[48] + mi := &file_rpc_gen_core_proto_msgTypes[50] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3653,7 +3860,7 @@ func (x *EntrypointOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use EntrypointOptions.ProtoReflect.Descriptor instead. func (*EntrypointOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{48} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{50} } func (x *EntrypointOptions) GetName() string { @@ -3733,6 +3940,7 @@ func (x *EntrypointOptions) GetCommands() []string { return nil } +// deprecated type ResourceOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3752,7 +3960,7 @@ type ResourceOptions struct { func (x *ResourceOptions) Reset() { *x = ResourceOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[49] + mi := &file_rpc_gen_core_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3765,7 +3973,7 @@ func (x *ResourceOptions) String() string { func (*ResourceOptions) ProtoMessage() {} func (x *ResourceOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[49] + mi := &file_rpc_gen_core_proto_msgTypes[51] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3778,7 +3986,7 @@ func (x *ResourceOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use ResourceOptions.ProtoReflect.Descriptor instead. func (*ResourceOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{49} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{51} } func (x *ResourceOptions) GetCpuQuotaLimit() float64 { @@ -3844,6 +4052,7 @@ func (x *ResourceOptions) GetVolumesRequest() []string { return nil } +// deprecated type Resource struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3865,7 +4074,7 @@ type Resource struct { func (x *Resource) Reset() { *x = Resource{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[50] + mi := &file_rpc_gen_core_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3878,7 +4087,7 @@ func (x *Resource) String() string { func (*Resource) ProtoMessage() {} func (x *Resource) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[50] + mi := &file_rpc_gen_core_proto_msgTypes[52] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3891,7 +4100,7 @@ func (x *Resource) ProtoReflect() protoreflect.Message { // Deprecated: Use Resource.ProtoReflect.Descriptor instead. func (*Resource) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{50} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{52} } func (x *Resource) GetCpuQuotaLimit() float64 { @@ -3982,7 +4191,7 @@ type Volume struct { func (x *Volume) Reset() { *x = Volume{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[51] + mi := &file_rpc_gen_core_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3995,7 +4204,7 @@ func (x *Volume) String() string { func (*Volume) ProtoMessage() {} func (x *Volume) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[51] + mi := &file_rpc_gen_core_proto_msgTypes[53] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4008,7 +4217,7 @@ func (x *Volume) ProtoReflect() protoreflect.Message { // Deprecated: Use Volume.ProtoReflect.Descriptor instead. func (*Volume) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{51} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{53} } func (x *Volume) GetVolume() map[string]int64 { @@ -4023,39 +4232,40 @@ type DeployOptions struct { 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"` - 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=open_stdin,json=openStdin,proto3" json:"open_stdin,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"` - NodeFilter *NodeFilter `protobuf:"bytes,25,opt,name=node_filter,json=nodeFilter,proto3" json:"node_filter,omitempty"` + 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"` + 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=open_stdin,json=openStdin,proto3" json:"open_stdin,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"` + OldResourceOpts *ResourceOptions `protobuf:"bytes,24,opt,name=old_resource_opts,json=oldResourceOpts,proto3" json:"old_resource_opts,omitempty"` + NodeFilter *NodeFilter `protobuf:"bytes,25,opt,name=node_filter,json=nodeFilter,proto3" json:"node_filter,omitempty"` // should be part of field no.19 - Modes map[string]*FileMode `protobuf:"bytes,26,rep,name=modes,proto3" json:"modes,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - Owners map[string]*FileOwner `protobuf:"bytes,27,rep,name=owners,proto3" json:"owners,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Modes map[string]*FileMode `protobuf:"bytes,26,rep,name=modes,proto3" json:"modes,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Owners map[string]*FileOwner `protobuf:"bytes,27,rep,name=owners,proto3" json:"owners,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ResourceOpts map[string]*RawParam `protobuf:"bytes,28,rep,name=resource_opts,json=resourceOpts,proto3" json:"resource_opts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *DeployOptions) Reset() { *x = DeployOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[52] + mi := &file_rpc_gen_core_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4068,7 +4278,7 @@ func (x *DeployOptions) String() string { func (*DeployOptions) ProtoMessage() {} func (x *DeployOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[52] + mi := &file_rpc_gen_core_proto_msgTypes[54] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4081,7 +4291,7 @@ func (x *DeployOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use DeployOptions.ProtoReflect.Descriptor instead. func (*DeployOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{52} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{54} } func (x *DeployOptions) GetName() string { @@ -4238,9 +4448,9 @@ func (x *DeployOptions) GetRawArgs() []byte { return nil } -func (x *DeployOptions) GetResourceOpts() *ResourceOptions { +func (x *DeployOptions) GetOldResourceOpts() *ResourceOptions { if x != nil { - return x.ResourceOpts + return x.OldResourceOpts } return nil } @@ -4266,6 +4476,13 @@ func (x *DeployOptions) GetOwners() map[string]*FileOwner { return nil } +func (x *DeployOptions) GetResourceOpts() map[string]*RawParam { + if x != nil { + return x.ResourceOpts + } + return nil +} + type ReplaceOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4281,7 +4498,7 @@ type ReplaceOptions struct { func (x *ReplaceOptions) Reset() { *x = ReplaceOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[53] + mi := &file_rpc_gen_core_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4294,7 +4511,7 @@ func (x *ReplaceOptions) String() string { func (*ReplaceOptions) ProtoMessage() {} func (x *ReplaceOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[53] + mi := &file_rpc_gen_core_proto_msgTypes[55] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4307,7 +4524,7 @@ func (x *ReplaceOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use ReplaceOptions.ProtoReflect.Descriptor instead. func (*ReplaceOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{53} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{55} } func (x *ReplaceOptions) GetDeployOpt() *DeployOptions { @@ -4359,7 +4576,7 @@ type CacheImageOptions struct { func (x *CacheImageOptions) Reset() { *x = CacheImageOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[54] + mi := &file_rpc_gen_core_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4372,7 +4589,7 @@ func (x *CacheImageOptions) String() string { func (*CacheImageOptions) ProtoMessage() {} func (x *CacheImageOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[54] + mi := &file_rpc_gen_core_proto_msgTypes[56] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4385,7 +4602,7 @@ func (x *CacheImageOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use CacheImageOptions.ProtoReflect.Descriptor instead. func (*CacheImageOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{54} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{56} } func (x *CacheImageOptions) GetPodname() string { @@ -4431,7 +4648,7 @@ type RemoveImageOptions struct { func (x *RemoveImageOptions) Reset() { *x = RemoveImageOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[55] + mi := &file_rpc_gen_core_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4444,7 +4661,7 @@ func (x *RemoveImageOptions) String() string { func (*RemoveImageOptions) ProtoMessage() {} func (x *RemoveImageOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[55] + mi := &file_rpc_gen_core_proto_msgTypes[57] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4457,7 +4674,7 @@ func (x *RemoveImageOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveImageOptions.ProtoReflect.Descriptor instead. func (*RemoveImageOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{55} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{57} } func (x *RemoveImageOptions) GetPodname() string { @@ -4508,7 +4725,7 @@ type ListImageOptions struct { func (x *ListImageOptions) Reset() { *x = ListImageOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[56] + mi := &file_rpc_gen_core_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4521,7 +4738,7 @@ func (x *ListImageOptions) String() string { func (*ListImageOptions) ProtoMessage() {} func (x *ListImageOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[56] + mi := &file_rpc_gen_core_proto_msgTypes[58] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4534,7 +4751,7 @@ func (x *ListImageOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use ListImageOptions.ProtoReflect.Descriptor instead. func (*ListImageOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{56} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{58} } func (x *ListImageOptions) GetPodname() string { @@ -4569,7 +4786,7 @@ type CopyPaths struct { func (x *CopyPaths) Reset() { *x = CopyPaths{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[57] + mi := &file_rpc_gen_core_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4582,7 +4799,7 @@ func (x *CopyPaths) String() string { func (*CopyPaths) ProtoMessage() {} func (x *CopyPaths) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[57] + mi := &file_rpc_gen_core_proto_msgTypes[59] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4595,7 +4812,7 @@ func (x *CopyPaths) ProtoReflect() protoreflect.Message { // Deprecated: Use CopyPaths.ProtoReflect.Descriptor instead. func (*CopyPaths) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{57} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{59} } func (x *CopyPaths) GetPaths() []string { @@ -4616,7 +4833,7 @@ type CopyOptions struct { func (x *CopyOptions) Reset() { *x = CopyOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[58] + mi := &file_rpc_gen_core_proto_msgTypes[60] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4629,7 +4846,7 @@ func (x *CopyOptions) String() string { func (*CopyOptions) ProtoMessage() {} func (x *CopyOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[58] + mi := &file_rpc_gen_core_proto_msgTypes[60] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4642,7 +4859,7 @@ func (x *CopyOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use CopyOptions.ProtoReflect.Descriptor instead. func (*CopyOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{58} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{60} } func (x *CopyOptions) GetTargets() map[string]*CopyPaths { @@ -4664,7 +4881,7 @@ type FileOwner struct { func (x *FileOwner) Reset() { *x = FileOwner{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[59] + mi := &file_rpc_gen_core_proto_msgTypes[61] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4677,7 +4894,7 @@ func (x *FileOwner) String() string { func (*FileOwner) ProtoMessage() {} func (x *FileOwner) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[59] + mi := &file_rpc_gen_core_proto_msgTypes[61] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4690,7 +4907,7 @@ func (x *FileOwner) ProtoReflect() protoreflect.Message { // Deprecated: Use FileOwner.ProtoReflect.Descriptor instead. func (*FileOwner) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{59} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{61} } func (x *FileOwner) GetUid() int32 { @@ -4718,7 +4935,7 @@ type FileMode struct { func (x *FileMode) Reset() { *x = FileMode{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[60] + mi := &file_rpc_gen_core_proto_msgTypes[62] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4731,7 +4948,7 @@ func (x *FileMode) String() string { func (*FileMode) ProtoMessage() {} func (x *FileMode) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[60] + mi := &file_rpc_gen_core_proto_msgTypes[62] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4744,7 +4961,7 @@ func (x *FileMode) ProtoReflect() protoreflect.Message { // Deprecated: Use FileMode.ProtoReflect.Descriptor instead. func (*FileMode) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{60} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{62} } func (x *FileMode) GetMode() int64 { @@ -4768,7 +4985,7 @@ type SendOptions struct { func (x *SendOptions) Reset() { *x = SendOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[61] + mi := &file_rpc_gen_core_proto_msgTypes[63] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4781,7 +4998,7 @@ func (x *SendOptions) String() string { func (*SendOptions) ProtoMessage() {} func (x *SendOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[61] + mi := &file_rpc_gen_core_proto_msgTypes[63] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4794,7 +5011,7 @@ func (x *SendOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use SendOptions.ProtoReflect.Descriptor instead. func (*SendOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{61} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{63} } func (x *SendOptions) GetIds() []string { @@ -4837,7 +5054,7 @@ type ErrorDetail struct { func (x *ErrorDetail) Reset() { *x = ErrorDetail{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[62] + mi := &file_rpc_gen_core_proto_msgTypes[64] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4850,7 +5067,7 @@ func (x *ErrorDetail) String() string { func (*ErrorDetail) ProtoMessage() {} func (x *ErrorDetail) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[62] + mi := &file_rpc_gen_core_proto_msgTypes[64] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4863,7 +5080,7 @@ func (x *ErrorDetail) ProtoReflect() protoreflect.Message { // Deprecated: Use ErrorDetail.ProtoReflect.Descriptor instead. func (*ErrorDetail) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{62} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{64} } func (x *ErrorDetail) GetCode() int64 { @@ -4896,7 +5113,7 @@ type BuildImageMessage struct { func (x *BuildImageMessage) Reset() { *x = BuildImageMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[63] + mi := &file_rpc_gen_core_proto_msgTypes[65] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4909,7 +5126,7 @@ func (x *BuildImageMessage) String() string { func (*BuildImageMessage) ProtoMessage() {} func (x *BuildImageMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[63] + mi := &file_rpc_gen_core_proto_msgTypes[65] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4922,7 +5139,7 @@ func (x *BuildImageMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use BuildImageMessage.ProtoReflect.Descriptor instead. func (*BuildImageMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{63} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{65} } func (x *BuildImageMessage) GetId() string { @@ -4972,21 +5189,22 @@ type CreateWorkloadMessage struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - 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"` + 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"` + ResourceArgs string `protobuf:"bytes,10,opt,name=resource_args,json=resourceArgs,proto3" json:"resource_args,omitempty"` } func (x *CreateWorkloadMessage) Reset() { *x = CreateWorkloadMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[64] + mi := &file_rpc_gen_core_proto_msgTypes[66] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4999,7 +5217,7 @@ func (x *CreateWorkloadMessage) String() string { func (*CreateWorkloadMessage) ProtoMessage() {} func (x *CreateWorkloadMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[64] + mi := &file_rpc_gen_core_proto_msgTypes[66] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5012,7 +5230,7 @@ func (x *CreateWorkloadMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateWorkloadMessage.ProtoReflect.Descriptor instead. func (*CreateWorkloadMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{64} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{66} } func (x *CreateWorkloadMessage) GetPodname() string { @@ -5078,6 +5296,13 @@ func (x *CreateWorkloadMessage) GetResource() *Resource { return nil } +func (x *CreateWorkloadMessage) GetResourceArgs() string { + if x != nil { + return x.ResourceArgs + } + return "" +} + type ReplaceWorkloadMessage struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5091,7 +5316,7 @@ type ReplaceWorkloadMessage struct { func (x *ReplaceWorkloadMessage) Reset() { *x = ReplaceWorkloadMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[65] + mi := &file_rpc_gen_core_proto_msgTypes[67] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5104,7 +5329,7 @@ func (x *ReplaceWorkloadMessage) String() string { func (*ReplaceWorkloadMessage) ProtoMessage() {} func (x *ReplaceWorkloadMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[65] + mi := &file_rpc_gen_core_proto_msgTypes[67] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5117,7 +5342,7 @@ func (x *ReplaceWorkloadMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use ReplaceWorkloadMessage.ProtoReflect.Descriptor instead. func (*ReplaceWorkloadMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{65} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{67} } func (x *ReplaceWorkloadMessage) GetCreate() *CreateWorkloadMessage { @@ -5155,7 +5380,7 @@ type CacheImageMessage struct { func (x *CacheImageMessage) Reset() { *x = CacheImageMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[66] + mi := &file_rpc_gen_core_proto_msgTypes[68] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5168,7 +5393,7 @@ func (x *CacheImageMessage) String() string { func (*CacheImageMessage) ProtoMessage() {} func (x *CacheImageMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[66] + mi := &file_rpc_gen_core_proto_msgTypes[68] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5181,7 +5406,7 @@ func (x *CacheImageMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use CacheImageMessage.ProtoReflect.Descriptor instead. func (*CacheImageMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{66} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{68} } func (x *CacheImageMessage) GetImage() string { @@ -5225,7 +5450,7 @@ type RemoveImageMessage struct { func (x *RemoveImageMessage) Reset() { *x = RemoveImageMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[67] + mi := &file_rpc_gen_core_proto_msgTypes[69] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5238,7 +5463,7 @@ func (x *RemoveImageMessage) String() string { func (*RemoveImageMessage) ProtoMessage() {} func (x *RemoveImageMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[67] + mi := &file_rpc_gen_core_proto_msgTypes[69] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5251,7 +5476,7 @@ func (x *RemoveImageMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveImageMessage.ProtoReflect.Descriptor instead. func (*RemoveImageMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{67} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{69} } func (x *RemoveImageMessage) GetImage() string { @@ -5287,7 +5512,7 @@ type ImageItem struct { func (x *ImageItem) Reset() { *x = ImageItem{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[68] + mi := &file_rpc_gen_core_proto_msgTypes[70] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5300,7 +5525,7 @@ func (x *ImageItem) String() string { func (*ImageItem) ProtoMessage() {} func (x *ImageItem) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[68] + mi := &file_rpc_gen_core_proto_msgTypes[70] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5313,7 +5538,7 @@ func (x *ImageItem) ProtoReflect() protoreflect.Message { // Deprecated: Use ImageItem.ProtoReflect.Descriptor instead. func (*ImageItem) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{68} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{70} } func (x *ImageItem) GetId() string { @@ -5343,7 +5568,7 @@ type ListImageMessage struct { func (x *ListImageMessage) Reset() { *x = ListImageMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[69] + mi := &file_rpc_gen_core_proto_msgTypes[71] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5356,7 +5581,7 @@ func (x *ListImageMessage) String() string { func (*ListImageMessage) ProtoMessage() {} func (x *ListImageMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[69] + mi := &file_rpc_gen_core_proto_msgTypes[71] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5369,7 +5594,7 @@ func (x *ListImageMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use ListImageMessage.ProtoReflect.Descriptor instead. func (*ListImageMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{69} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{71} } func (x *ListImageMessage) GetImages() []*ImageItem { @@ -5406,7 +5631,7 @@ type RemoveWorkloadMessage struct { func (x *RemoveWorkloadMessage) Reset() { *x = RemoveWorkloadMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[70] + mi := &file_rpc_gen_core_proto_msgTypes[72] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5419,7 +5644,7 @@ func (x *RemoveWorkloadMessage) String() string { func (*RemoveWorkloadMessage) ProtoMessage() {} func (x *RemoveWorkloadMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[70] + mi := &file_rpc_gen_core_proto_msgTypes[72] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5432,7 +5657,7 @@ func (x *RemoveWorkloadMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveWorkloadMessage.ProtoReflect.Descriptor instead. func (*RemoveWorkloadMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{70} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{72} } func (x *RemoveWorkloadMessage) GetId() string { @@ -5468,7 +5693,7 @@ type DissociateWorkloadMessage struct { func (x *DissociateWorkloadMessage) Reset() { *x = DissociateWorkloadMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[71] + mi := &file_rpc_gen_core_proto_msgTypes[73] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5481,7 +5706,7 @@ func (x *DissociateWorkloadMessage) String() string { func (*DissociateWorkloadMessage) ProtoMessage() {} func (x *DissociateWorkloadMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[71] + mi := &file_rpc_gen_core_proto_msgTypes[73] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5494,7 +5719,7 @@ func (x *DissociateWorkloadMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use DissociateWorkloadMessage.ProtoReflect.Descriptor instead. func (*DissociateWorkloadMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{71} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{73} } func (x *DissociateWorkloadMessage) GetId() string { @@ -5522,7 +5747,7 @@ type ReallocResourceMessage struct { func (x *ReallocResourceMessage) Reset() { *x = ReallocResourceMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[72] + mi := &file_rpc_gen_core_proto_msgTypes[74] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5535,7 +5760,7 @@ func (x *ReallocResourceMessage) String() string { func (*ReallocResourceMessage) ProtoMessage() {} func (x *ReallocResourceMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[72] + mi := &file_rpc_gen_core_proto_msgTypes[74] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5548,7 +5773,7 @@ func (x *ReallocResourceMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use ReallocResourceMessage.ProtoReflect.Descriptor instead. func (*ReallocResourceMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{72} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{74} } func (x *ReallocResourceMessage) GetError() string { @@ -5573,7 +5798,7 @@ type CopyMessage struct { func (x *CopyMessage) Reset() { *x = CopyMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[73] + mi := &file_rpc_gen_core_proto_msgTypes[75] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5586,7 +5811,7 @@ func (x *CopyMessage) String() string { func (*CopyMessage) ProtoMessage() {} func (x *CopyMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[73] + mi := &file_rpc_gen_core_proto_msgTypes[75] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5599,7 +5824,7 @@ func (x *CopyMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use CopyMessage.ProtoReflect.Descriptor instead. func (*CopyMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{73} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{75} } func (x *CopyMessage) GetId() string { @@ -5650,7 +5875,7 @@ type SendMessage struct { func (x *SendMessage) Reset() { *x = SendMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[74] + mi := &file_rpc_gen_core_proto_msgTypes[76] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5663,7 +5888,7 @@ func (x *SendMessage) String() string { func (*SendMessage) ProtoMessage() {} func (x *SendMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[74] + mi := &file_rpc_gen_core_proto_msgTypes[76] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5676,7 +5901,7 @@ func (x *SendMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use SendMessage.ProtoReflect.Descriptor instead. func (*SendMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{74} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{76} } func (x *SendMessage) GetId() string { @@ -5713,7 +5938,7 @@ type AttachWorkloadMessage struct { func (x *AttachWorkloadMessage) Reset() { *x = AttachWorkloadMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[75] + mi := &file_rpc_gen_core_proto_msgTypes[77] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5726,7 +5951,7 @@ func (x *AttachWorkloadMessage) String() string { func (*AttachWorkloadMessage) ProtoMessage() {} func (x *AttachWorkloadMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[75] + mi := &file_rpc_gen_core_proto_msgTypes[77] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5739,7 +5964,7 @@ func (x *AttachWorkloadMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use AttachWorkloadMessage.ProtoReflect.Descriptor instead. func (*AttachWorkloadMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{75} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{77} } func (x *AttachWorkloadMessage) GetWorkloadId() string { @@ -5777,7 +6002,7 @@ type RunAndWaitOptions struct { func (x *RunAndWaitOptions) Reset() { *x = RunAndWaitOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[76] + mi := &file_rpc_gen_core_proto_msgTypes[78] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5790,7 +6015,7 @@ func (x *RunAndWaitOptions) String() string { func (*RunAndWaitOptions) ProtoMessage() {} func (x *RunAndWaitOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[76] + mi := &file_rpc_gen_core_proto_msgTypes[78] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5803,7 +6028,7 @@ func (x *RunAndWaitOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use RunAndWaitOptions.ProtoReflect.Descriptor instead. func (*RunAndWaitOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{76} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{78} } func (x *RunAndWaitOptions) GetDeployOptions() *DeployOptions { @@ -5847,7 +6072,7 @@ type ControlWorkloadOptions struct { func (x *ControlWorkloadOptions) Reset() { *x = ControlWorkloadOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[77] + mi := &file_rpc_gen_core_proto_msgTypes[79] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5860,7 +6085,7 @@ func (x *ControlWorkloadOptions) String() string { func (*ControlWorkloadOptions) ProtoMessage() {} func (x *ControlWorkloadOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[77] + mi := &file_rpc_gen_core_proto_msgTypes[79] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5873,7 +6098,7 @@ func (x *ControlWorkloadOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use ControlWorkloadOptions.ProtoReflect.Descriptor instead. func (*ControlWorkloadOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{77} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{79} } func (x *ControlWorkloadOptions) GetIds() []string { @@ -5910,7 +6135,7 @@ type ControlWorkloadMessage struct { func (x *ControlWorkloadMessage) Reset() { *x = ControlWorkloadMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[78] + mi := &file_rpc_gen_core_proto_msgTypes[80] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5923,7 +6148,7 @@ func (x *ControlWorkloadMessage) String() string { func (*ControlWorkloadMessage) ProtoMessage() {} func (x *ControlWorkloadMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[78] + mi := &file_rpc_gen_core_proto_msgTypes[80] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5936,7 +6161,7 @@ func (x *ControlWorkloadMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use ControlWorkloadMessage.ProtoReflect.Descriptor instead. func (*ControlWorkloadMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{78} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{80} } func (x *ControlWorkloadMessage) GetId() string { @@ -5975,7 +6200,7 @@ type LogStreamOptions struct { func (x *LogStreamOptions) Reset() { *x = LogStreamOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[79] + mi := &file_rpc_gen_core_proto_msgTypes[81] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5988,7 +6213,7 @@ func (x *LogStreamOptions) String() string { func (*LogStreamOptions) ProtoMessage() {} func (x *LogStreamOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[79] + mi := &file_rpc_gen_core_proto_msgTypes[81] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6001,7 +6226,7 @@ func (x *LogStreamOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use LogStreamOptions.ProtoReflect.Descriptor instead. func (*LogStreamOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{79} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{81} } func (x *LogStreamOptions) GetId() string { @@ -6053,7 +6278,7 @@ type LogStreamMessage struct { func (x *LogStreamMessage) Reset() { *x = LogStreamMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[80] + mi := &file_rpc_gen_core_proto_msgTypes[82] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6066,7 +6291,7 @@ func (x *LogStreamMessage) String() string { func (*LogStreamMessage) ProtoMessage() {} func (x *LogStreamMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[80] + mi := &file_rpc_gen_core_proto_msgTypes[82] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6079,7 +6304,7 @@ func (x *LogStreamMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use LogStreamMessage.ProtoReflect.Descriptor instead. func (*LogStreamMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{80} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{82} } func (x *LogStreamMessage) GetId() string { @@ -6126,7 +6351,7 @@ type ExecuteWorkloadOptions struct { func (x *ExecuteWorkloadOptions) Reset() { *x = ExecuteWorkloadOptions{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[81] + mi := &file_rpc_gen_core_proto_msgTypes[83] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6139,7 +6364,7 @@ func (x *ExecuteWorkloadOptions) String() string { func (*ExecuteWorkloadOptions) ProtoMessage() {} func (x *ExecuteWorkloadOptions) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[81] + mi := &file_rpc_gen_core_proto_msgTypes[83] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6152,7 +6377,7 @@ func (x *ExecuteWorkloadOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecuteWorkloadOptions.ProtoReflect.Descriptor instead. func (*ExecuteWorkloadOptions) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{81} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{83} } func (x *ExecuteWorkloadOptions) GetWorkloadId() string { @@ -6209,7 +6434,7 @@ type CapacityMessage struct { func (x *CapacityMessage) Reset() { *x = CapacityMessage{} if protoimpl.UnsafeEnabled { - mi := &file_rpc_gen_core_proto_msgTypes[82] + mi := &file_rpc_gen_core_proto_msgTypes[84] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6222,7 +6447,7 @@ func (x *CapacityMessage) String() string { func (*CapacityMessage) ProtoMessage() {} func (x *CapacityMessage) ProtoReflect() protoreflect.Message { - mi := &file_rpc_gen_core_proto_msgTypes[82] + mi := &file_rpc_gen_core_proto_msgTypes[84] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6235,7 +6460,7 @@ func (x *CapacityMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use CapacityMessage.ProtoReflect.Descriptor instead. func (*CapacityMessage) Descriptor() ([]byte, []int) { - return file_rpc_gen_core_proto_rawDescGZIP(), []int{82} + return file_rpc_gen_core_proto_rawDescGZIP(), []int{84} } func (x *CapacityMessage) GetTotal() int64 { @@ -6303,383 +6528,436 @@ var file_rpc_gen_core_proto_rawDesc = []byte{ 0x0e, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x0d, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x52, 0x65, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, 0xd0, 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, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x70, - 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x04, 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, 0x05, - 0x20, 0x01, 0x28, 0x01, 0x52, 0x0d, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x50, 0x65, 0x72, 0x63, - 0x65, 0x6e, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x69, 0x66, 0x66, 0x73, 0x18, 0x06, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x05, 0x64, 0x69, 0x66, 0x66, 0x73, 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, 0x8d, 0x0a, 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, 0x12, 0x46, 0x0a, 0x10, 0x69, 0x6e, 0x69, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x61, 0x5f, - 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x15, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, - 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4e, 0x75, 0x6d, 0x61, 0x4d, - 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x69, 0x6e, 0x69, 0x74, - 0x4e, 0x75, 0x6d, 0x61, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x79, - 0x70, 0x61, 0x73, 0x73, 0x18, 0x16, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x62, 0x79, 0x70, 0x61, - 0x73, 0x73, 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, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, 0x23, 0x0a, 0x0b, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, + 0x53, 0x6c, 0x69, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x22, 0x5d, 0x0a, 0x08, 0x52, + 0x61, 0x77, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x12, 0x12, 0x0a, 0x03, 0x73, 0x74, 0x72, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x03, 0x73, 0x74, 0x72, 0x12, 0x34, 0x0a, 0x0c, 0x73, + 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x6c, 0x69, + 0x63, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x6c, 0x69, 0x63, + 0x65, 0x42, 0x07, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xa4, 0x02, 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, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, + 0x67, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x04, 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, 0x05, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0d, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, + 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x69, 0x66, 0x66, 0x73, + 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x64, 0x69, 0x66, 0x66, 0x73, 0x12, 0x2b, 0x0a, + 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, + 0x74, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x75, 0x73, 0x61, 0x67, 0x65, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x55, 0x73, 0x61, 0x67, + 0x65, 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, 0xe1, + 0x0a, 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, 0x12, 0x46, 0x0a, 0x10, 0x69, 0x6e, + 0x69, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x61, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x15, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x2e, 0x49, + 0x6e, 0x69, 0x74, 0x4e, 0x75, 0x6d, 0x61, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x0e, 0x69, 0x6e, 0x69, 0x74, 0x4e, 0x75, 0x6d, 0x61, 0x4d, 0x65, 0x6d, 0x6f, + 0x72, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x79, 0x70, 0x61, 0x73, 0x73, 0x18, 0x16, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x06, 0x62, 0x79, 0x70, 0x61, 0x73, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, + 0x17, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, + 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x5f, 0x75, 0x73, 0x61, 0x67, 0x65, 0x18, 0x18, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x55, 0x73, 0x61, 0x67, 0x65, 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, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3a, 0x0a, 0x0c, 0x49, 0x6e, 0x69, 0x74, 0x43, 0x70, 0x75, + 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, 0x1a, + 0x41, 0x0a, 0x13, 0x49, 0x6e, 0x69, 0x74, 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, 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, 0x08, 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, 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, + 0x25, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x5f, 0x64, 0x6f, 0x77, + 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, + 0x64, 0x73, 0x44, 0x6f, 0x77, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x12, 0x29, 0x0a, 0x0a, 0x62, 0x79, 0x70, 0x61, 0x73, 0x73, 0x5f, 0x6f, 0x70, 0x74, + 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x54, 0x72, 0x69, 0x4f, + 0x70, 0x74, 0x52, 0x09, 0x62, 0x79, 0x70, 0x61, 0x73, 0x73, 0x4f, 0x70, 0x74, 0x12, 0x0e, 0x0a, + 0x02, 0x63, 0x61, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x63, 0x61, 0x12, 0x12, 0x0a, + 0x04, 0x63, 0x65, 0x72, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x65, 0x72, + 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x49, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, + 0x6f, 0x70, 0x74, 0x73, 0x18, 0x10, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x62, 0x2e, + 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x73, 0x12, 0x14, + 0x0a, 0x05, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x64, + 0x65, 0x6c, 0x74, 0x61, 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, 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, + 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, 0x1a, 0x3d, 0x0a, 0x0f, 0x49, 0x6e, 0x69, - 0x74, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x4d, 0x0a, 0x11, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 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, 0x22, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x61, 0x77, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x44, 0x0a, 0x14, 0x53, 0x65, 0x74, 0x4e, + 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 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, 0x10, 0x0a, 0x03, + 0x74, 0x74, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x74, 0x74, 0x6c, 0x22, 0x32, + 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 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, 0x7b, 0x0a, 0x17, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 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, 0x12, 0x14, 0x0a, 0x05, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x05, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, + 0xc5, 0x01, 0x0a, 0x0a, 0x4e, 0x6f, 0x64, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x1a, + 0x0a, 0x08, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x08, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x65, 0x78, + 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x65, 0x78, + 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, + 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x61, 0x6c, + 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x61, 0x6c, 0x6c, 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, 0xa6, 0x04, 0x0a, 0x08, 0x57, 0x6f, 0x72, 0x6b, + 0x6c, 0x6f, 0x61, 0x64, 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, 0x30, + 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, + 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x4c, 0x61, 0x62, + 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, + 0x12, 0x33, 0x0a, 0x07, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x18, 0x07, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x2a, 0x0a, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x62, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, + 0x6d, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x76, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x03, 0x65, 0x6e, 0x76, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x61, 0x72, 0x67, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x72, 0x67, 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, 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, 0xd5, 0x02, 0x0a, 0x0e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x3c, 0x0a, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, + 0x72, 0x6b, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x62, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x70, 0x70, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x6e, 0x61, 0x6d, 0x65, 0x12, + 0x1a, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 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, 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, 0x1a, 0x41, 0x0a, 0x13, 0x49, 0x6e, 0x69, 0x74, 0x4e, 0x75, 0x6d, 0x61, 0x4d, - 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 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, 0x3d, 0x0a, 0x0f, 0x57, 0x6f, 0x72, 0x6b, + 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x62, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x47, 0x0a, 0x19, 0x53, 0x65, 0x74, 0x57, 0x6f, + 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, + 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x22, 0xf3, 0x01, 0x0a, 0x1b, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x43, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, + 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x6c, 0x6f, 0x61, 0x64, 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, 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, 0x93, 0x07, 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, 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, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xb1, 0x01, 0x0a, 0x1b, 0x57, 0x6f, 0x72, 0x6b, 0x6c, + 0x6f, 0x61, 0x64, 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, 0x28, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, + 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, + 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x37, 0x0a, 0x09, 0x57, 0x6f, + 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x12, 0x2a, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x6c, + 0x6f, 0x61, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, + 0x61, 0x64, 0x73, 0x22, 0x1c, 0x0a, 0x0a, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, + 0x44, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, + 0x64, 0x22, 0x1f, 0x0a, 0x0b, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x44, 0x73, + 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, + 0x64, 0x73, 0x22, 0x53, 0x0a, 0x15, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, + 0x6c, 0x6f, 0x61, 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, 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, 0x2d, 0x0a, 0x19, 0x44, 0x69, 0x73, 0x73, 0x6f, + 0x63, 0x69, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 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, 0x22, 0xa9, 0x02, 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, 0x3f, 0x0a, 0x11, 0x6f, 0x6c, 0x64, 0x5f, 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, 0x0f, 0x6f, 0x6c, 0x64, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x73, 0x12, 0x49, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x24, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, + 0x70, 0x74, 0x73, 0x1a, 0x4d, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, + 0x70, 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, 0x22, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x52, + 0x61, 0x77, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 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, 0xee, 0x06, 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, 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, 0x25, 0x0a, 0x0e, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, - 0x5f, 0x64, 0x6f, 0x77, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x77, 0x6f, 0x72, - 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x44, 0x6f, 0x77, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x65, 0x6e, - 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x6e, - 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x29, 0x0a, 0x0a, 0x62, 0x79, 0x70, 0x61, 0x73, 0x73, - 0x5f, 0x6f, 0x70, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x70, 0x62, 0x2e, - 0x54, 0x72, 0x69, 0x4f, 0x70, 0x74, 0x52, 0x09, 0x62, 0x79, 0x70, 0x61, 0x73, 0x73, 0x4f, 0x70, - 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x63, 0x61, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x63, - 0x61, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x72, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x63, 0x65, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x0f, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 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, + 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, 0x12, 0x49, + 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x73, 0x18, + 0x0f, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x64, 0x64, 0x4e, 0x6f, + 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x4f, 0x70, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x72, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, 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, 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, 0x44, 0x0a, 0x14, - 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 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, 0x10, 0x0a, 0x03, 0x74, 0x74, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x74, - 0x74, 0x6c, 0x22, 0x32, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 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, 0x7b, 0x0a, 0x17, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 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, 0x12, 0x14, 0x0a, 0x05, 0x61, 0x6c, 0x69, 0x76, 0x65, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x22, 0xc5, 0x01, 0x0a, 0x0a, 0x4e, 0x6f, 0x64, 0x65, 0x46, 0x69, 0x6c, 0x74, - 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73, 0x12, 0x1a, - 0x0a, 0x08, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x08, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x06, 0x6c, 0x61, - 0x62, 0x65, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x62, 0x2e, - 0x4e, 0x6f, 0x64, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, - 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x10, - 0x0a, 0x03, 0x61, 0x6c, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x61, 0x6c, 0x6c, - 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, 0x81, 0x04, 0x0a, 0x08, - 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x30, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x06, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, - 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, - 0x62, 0x65, 0x6c, 0x73, 0x12, 0x33, 0x0a, 0x07, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x18, - 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, - 0x6f, 0x61, 0x64, 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, - 0x2a, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, - 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x76, 0x18, 0x0c, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x03, 0x65, 0x6e, 0x76, 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, - 0xd5, 0x02, 0x0a, 0x0e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x3c, 0x0a, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, - 0x6b, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, - 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x70, 0x70, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1a, - 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x65, 0x6e, - 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, - 0x65, 0x6e, 0x74, 0x72, 0x79, 0x70, 0x6f, 0x69, 0x6e, 0x74, 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, 0x3d, 0x0a, 0x0f, 0x57, 0x6f, 0x72, 0x6b, 0x6c, - 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x62, 0x2e, - 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, - 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x47, 0x0a, 0x19, 0x53, 0x65, 0x74, 0x57, 0x6f, 0x72, - 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x4f, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, - 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0xf3, 0x01, 0x0a, 0x1b, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x43, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, - 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, - 0x6f, 0x61, 0x64, 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, 0xb1, 0x01, 0x0a, 0x1b, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, - 0x61, 0x64, 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, 0x28, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, - 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, - 0x2a, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x37, 0x0a, 0x09, 0x57, 0x6f, 0x72, - 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x12, 0x2a, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, - 0x61, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x57, - 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, - 0x64, 0x73, 0x22, 0x1c, 0x0a, 0x0a, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x44, - 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, - 0x22, 0x1f, 0x0a, 0x0b, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x44, 0x73, 0x12, - 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, - 0x73, 0x22, 0x3f, 0x0a, 0x15, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, - 0x6f, 0x61, 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, 0x14, 0x0a, 0x05, - 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, - 0x63, 0x65, 0x22, 0x2d, 0x0a, 0x19, 0x44, 0x69, 0x73, 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, - 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 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, 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, + 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, - 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, + 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, 0x1a, 0x4d, 0x0a, 0x11, 0x52, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 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, 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, + 0x79, 0x12, 0x22, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x61, 0x77, 0x50, 0x61, 0x72, 0x61, 0x6d, 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, @@ -6916,7 +7194,7 @@ var file_rpc_gen_core_proto_rawDesc = []byte{ 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, 0xb1, 0x0b, 0x0a, 0x0d, 0x44, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd1, 0x0c, 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, @@ -6966,44 +7244,54 @@ var file_rpc_gen_core_proto_rawDesc = []byte{ 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, 0x12, - 0x2f, 0x0a, 0x0b, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x19, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x46, 0x69, - 0x6c, 0x74, 0x65, 0x72, 0x52, 0x0a, 0x6e, 0x6f, 0x64, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, - 0x12, 0x32, 0x0a, 0x05, 0x6d, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x1a, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x1c, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x2e, 0x4d, 0x6f, 0x64, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x6d, - 0x6f, 0x64, 0x65, 0x73, 0x12, 0x35, 0x0a, 0x06, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x73, 0x18, 0x1b, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4f, 0x77, 0x6e, 0x65, 0x72, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x06, 0x6f, 0x77, 0x6e, 0x65, 0x72, 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, + 0x77, 0x41, 0x72, 0x67, 0x73, 0x12, 0x3f, 0x0a, 0x11, 0x6f, 0x6c, 0x64, 0x5f, 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, 0x0f, 0x6f, 0x6c, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x4f, 0x70, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x0b, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x66, + 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x70, 0x62, + 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x0a, 0x6e, 0x6f, 0x64, + 0x65, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x32, 0x0a, 0x05, 0x6d, 0x6f, 0x64, 0x65, 0x73, + 0x18, 0x1a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, + 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4d, 0x6f, 0x64, 0x65, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x6d, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x35, 0x0a, 0x06, 0x6f, + 0x77, 0x6e, 0x65, 0x72, 0x73, 0x18, 0x1b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x62, + 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x4f, + 0x77, 0x6e, 0x65, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6f, 0x77, 0x6e, 0x65, + 0x72, 0x73, 0x12, 0x48, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6f, + 0x70, 0x74, 0x73, 0x18, 0x1c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x62, 0x2e, 0x44, + 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 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, 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, 0x1a, 0x46, 0x0a, 0x0a, 0x4d, - 0x6f, 0x64, 0x65, 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, 0x22, 0x0a, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, - 0x46, 0x69, 0x6c, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x1a, 0x48, 0x0a, 0x0b, 0x4f, 0x77, 0x6e, 0x65, 0x72, 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, 0x46, 0x69, 0x6c, 0x65, 0x4f, 0x77, 0x6e, - 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3f, 0x0a, + 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, 0x1a, 0x46, 0x0a, 0x0a, + 0x4d, 0x6f, 0x64, 0x65, 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, 0x22, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, + 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x48, 0x0a, 0x0b, 0x4f, 0x77, 0x6e, 0x65, 0x72, 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, 0x46, 0x69, 0x6c, 0x65, 0x4f, 0x77, + 0x6e, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x4d, + 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4f, 0x70, 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, 0x22, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x61, 0x77, 0x50, 0x61, 0x72, + 0x61, 0x6d, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3f, 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, @@ -7110,7 +7398,7 @@ var file_rpc_gen_core_proto_rawDesc = []byte{ 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, 0xdd, 0x02, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, + 0x74, 0x61, 0x69, 0x6c, 0x22, 0x82, 0x03, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, @@ -7128,316 +7416,318 @@ var file_rpc_gen_core_proto_rawDesc = []byte{ 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, 0x94, 0x01, 0x0a, 0x16, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, - 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, - 0x31, 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, - 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x06, 0x63, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x12, 0x31, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x57, 0x6f, - 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x2f, 0x0a, 0x09, 0x49, 0x6d, 0x61, 0x67, - 0x65, 0x49, 0x74, 0x65, 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x02, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x22, 0x67, 0x0a, 0x10, 0x4c, 0x69, 0x73, - 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, - 0x06, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, - 0x70, 0x62, 0x2e, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x49, 0x74, 0x65, 0x6d, 0x52, 0x06, 0x69, 0x6d, - 0x61, 0x67, 0x65, 0x73, 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, 0x10, 0x0a, 0x03, 0x65, 0x72, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x65, - 0x72, 0x72, 0x22, 0x55, 0x0a, 0x15, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, - 0x6c, 0x6f, 0x61, 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, 0x18, 0x0a, 0x07, 0x73, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x5f, 0x61, 0x72, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x72, 0x67, 0x73, 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, 0x94, 0x01, 0x0a, 0x16, 0x52, 0x65, + 0x70, 0x6c, 0x61, 0x63, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x12, 0x31, 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, + 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x31, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, + 0x6f, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x6f, 0x6b, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x6f, 0x6b, 0x22, 0x41, 0x0a, 0x19, 0x44, 0x69, 0x73, - 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 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, 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, 0x6f, 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, 0x12, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, - 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, - 0x61, 0x74, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, - 0x61, 0x18, 0x05, 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, 0x87, 0x01, 0x0a, 0x15, 0x41, 0x74, 0x74, 0x61, 0x63, - 0x68, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, - 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x39, 0x0a, 0x0f, 0x73, 0x74, 0x64, 0x5f, 0x73, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x11, - 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x79, 0x70, - 0x65, 0x52, 0x0d, 0x73, 0x74, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x79, 0x70, 0x65, - 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, 0x54, 0x0a, - 0x16, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 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, 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, 0x52, 0x0a, 0x16, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x57, 0x6f, - 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x0e, 0x0a, + 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, 0x2f, 0x0a, + 0x09, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x49, 0x74, 0x65, 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, + 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x22, 0x67, + 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x12, 0x25, 0x0a, 0x06, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x49, 0x74, 0x65, + 0x6d, 0x52, 0x06, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x73, 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, 0x10, 0x0a, 0x03, 0x65, 0x72, 0x72, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x65, 0x72, 0x72, 0x22, 0x55, 0x0a, 0x15, 0x52, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 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, 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, 0x41, + 0x0a, 0x19, 0x44, 0x69, 0x73, 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, + 0x6c, 0x6f, 0x61, 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, 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, 0x6f, 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, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, + 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 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, 0x87, 0x01, 0x0a, 0x15, + 0x41, 0x74, 0x74, 0x61, 0x63, 0x68, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, + 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, + 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x39, 0x0a, 0x0f, 0x73, 0x74, + 0x64, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x64, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, 0x73, 0x74, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x54, 0x79, 0x70, 0x65, 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, 0x54, 0x0a, 0x16, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x57, 0x6f, 0x72, + 0x6b, 0x6c, 0x6f, 0x61, 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, 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, 0x52, 0x0a, 0x16, 0x43, 0x6f, 0x6e, 0x74, + 0x72, 0x6f, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 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, 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, 0x7a, 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, + 0x12, 0x16, 0x0a, 0x06, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x06, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x22, 0x87, 0x01, 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, 0x68, 0x6f, 0x6f, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x04, 0x68, 0x6f, 0x6f, 0x6b, 0x22, 0x7a, 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, 0x12, 0x16, 0x0a, 0x06, 0x66, - 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x66, 0x6f, 0x6c, - 0x6c, 0x6f, 0x77, 0x22, 0x87, 0x01, 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, 0x12, 0x39, 0x0a, 0x0f, 0x73, 0x74, 0x64, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x11, 0x2e, 0x70, 0x62, - 0x2e, 0x53, 0x74, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, - 0x73, 0x74, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x79, 0x70, 0x65, 0x22, 0xbd, 0x01, - 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, - 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, - 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, - 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x22, 0xbc, 0x01, - 0x0a, 0x0f, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x05, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x12, 0x50, 0x0a, 0x0f, 0x6e, 0x6f, 0x64, 0x65, 0x5f, - 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x27, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, - 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x6e, 0x6f, 0x64, 0x65, 0x43, - 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x41, 0x0a, 0x13, 0x4e, 0x6f, 0x64, - 0x65, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x69, 0x65, 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, - 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 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, 0x2a, 0x52, 0x0a, 0x0d, 0x53, 0x74, 0x64, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x44, 0x4f, 0x55, 0x54, - 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x44, 0x45, 0x52, 0x52, 0x10, 0x01, 0x12, 0x12, - 0x0a, 0x0e, 0x54, 0x59, 0x50, 0x45, 0x57, 0x4f, 0x52, 0x4b, 0x4c, 0x4f, 0x41, 0x44, 0x49, 0x44, - 0x10, 0x06, 0x12, 0x15, 0x0a, 0x08, 0x45, 0x52, 0x55, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0xff, - 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x32, 0xf4, 0x14, 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, - 0x3c, 0x0a, 0x11, 0x50, 0x6f, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x12, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x6f, 0x64, - 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, 0x30, 0x01, 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, 0x34, 0x0a, - 0x0e, 0x50, 0x6f, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, + 0x72, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x39, 0x0a, 0x0f, 0x73, 0x74, 0x64, 0x5f, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x0d, 0x73, 0x74, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x79, + 0x70, 0x65, 0x22, 0xbd, 0x01, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x57, 0x6f, + 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1f, 0x0a, + 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x22, 0xbc, 0x01, 0x0a, 0x0f, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x12, 0x50, 0x0a, 0x0f, + 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x61, 0x70, 0x61, 0x63, + 0x69, 0x74, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x43, + 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, + 0x6e, 0x6f, 0x64, 0x65, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x41, + 0x0a, 0x13, 0x4e, 0x6f, 0x64, 0x65, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x69, 0x65, 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, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 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, 0x2a, 0x52, 0x0a, 0x0d, 0x53, 0x74, + 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x53, + 0x54, 0x44, 0x4f, 0x55, 0x54, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x44, 0x45, 0x52, + 0x52, 0x10, 0x01, 0x12, 0x12, 0x0a, 0x0e, 0x54, 0x59, 0x50, 0x45, 0x57, 0x4f, 0x52, 0x4b, 0x4c, + 0x4f, 0x41, 0x44, 0x49, 0x44, 0x10, 0x06, 0x12, 0x15, 0x0a, 0x08, 0x45, 0x52, 0x55, 0x45, 0x52, + 0x52, 0x4f, 0x52, 0x10, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x32, 0xf4, + 0x14, 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, 0x3c, 0x0a, 0x11, 0x50, 0x6f, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x47, + 0x65, 0x74, 0x50, 0x6f, 0x64, 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, + 0x30, 0x01, 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, 0x34, 0x0a, 0x0e, 0x50, 0x6f, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x12, 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x6f, + 0x64, 0x65, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x08, 0x2e, 0x70, 0x62, 0x2e, + 0x4e, 0x6f, 0x64, 0x65, 0x22, 0x00, 0x30, 0x01, 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, 0x31, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, + 0x67, 0x69, 0x6e, 0x65, 0x12, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, + 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6e, + 0x67, 0x69, 0x6e, 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, 0x30, 0x01, 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, 0x31, - 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x12, - 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x1a, 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6e, 0x67, 0x69, 0x6e, 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, 0x36, 0x0a, 0x0d, - 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x18, 0x2e, - 0x70, 0x62, 0x2e, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, - 0x74, 0x79, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x18, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, - 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, - 0x1b, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x12, 0x3e, - 0x0a, 0x10, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x12, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x1b, 0x2e, - 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 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, 0x3b, 0x0a, 0x11, 0x43, 0x61, 0x6c, 0x63, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x43, 0x61, - 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, - 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x43, - 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2d, - 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x0e, 0x2e, - 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x44, 0x1a, 0x0c, 0x2e, - 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x00, 0x12, 0x30, 0x0a, - 0x0c, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x12, 0x0f, 0x2e, - 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x44, 0x73, 0x1a, 0x0d, - 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x22, 0x00, 0x12, - 0x3b, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, - 0x12, 0x18, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, - 0x61, 0x64, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x0c, 0x2e, 0x70, 0x62, 0x2e, - 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x00, 0x30, 0x01, 0x12, 0x38, 0x0a, 0x11, - 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, - 0x73, 0x12, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x4f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, - 0x6f, 0x61, 0x64, 0x73, 0x22, 0x00, 0x12, 0x3c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, - 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0f, 0x2e, 0x70, - 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x44, 0x73, 0x1a, 0x13, 0x2e, - 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x22, 0x00, 0x12, 0x4a, 0x0a, 0x12, 0x53, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, - 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1d, 0x2e, 0x70, 0x62, 0x2e, - 0x53, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x57, - 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, - 0x12, 0x5c, 0x0a, 0x14, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x1f, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, - 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1f, 0x2e, 0x70, 0x62, 0x2e, 0x57, - 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x3b, 0x0a, - 0x09, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x12, 0x14, 0x2e, 0x70, 0x62, 0x2e, - 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x1a, 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x42, 0x0a, 0x0e, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x11, 0x2e, 0x70, - 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, - 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, - 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x45, - 0x0a, 0x0f, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, - 0x64, 0x12, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x4f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x61, - 0x63, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x4a, 0x0a, 0x0e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x57, - 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, - 0x6f, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x1a, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x57, 0x6f, - 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, - 0x01, 0x12, 0x56, 0x0a, 0x12, 0x44, 0x69, 0x73, 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x57, - 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1d, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x69, 0x73, - 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1d, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x69, 0x73, 0x73, - 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x4d, 0x0a, 0x0f, 0x43, 0x6f, 0x6e, - 0x74, 0x72, 0x6f, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1a, 0x2e, 0x70, - 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, - 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, - 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x4e, 0x0a, 0x0f, 0x45, 0x78, 0x65, 0x63, - 0x75, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1a, 0x2e, 0x70, 0x62, - 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x74, 0x74, - 0x61, 0x63, 0x68, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x44, 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, - 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x74, 0x74, 0x61, 0x63, 0x68, 0x57, 0x6f, 0x72, 0x6b, 0x6c, - 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, - 0x42, 0x28, 0x5a, 0x26, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, - 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x72, 0x75, 0x32, 0x2f, 0x63, 0x6f, 0x72, 0x65, 0x2f, - 0x72, 0x70, 0x63, 0x2f, 0x67, 0x65, 0x6e, 0x3b, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x00, 0x12, 0x36, 0x0a, 0x0d, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x12, 0x18, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x09, 0x2e, 0x70, + 0x62, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0d, 0x47, 0x65, 0x74, + 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x18, 0x2e, 0x70, 0x62, 0x2e, + 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1b, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x22, 0x00, 0x12, 0x3e, 0x0a, 0x10, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, + 0x74, 0x79, 0x1a, 0x1b, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, + 0x00, 0x30, 0x01, 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, 0x3b, 0x0a, 0x11, 0x43, 0x61, 0x6c, 0x63, 0x75, 0x6c, + 0x61, 0x74, 0x65, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x11, 0x2e, 0x70, 0x62, + 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x13, + 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x12, 0x2d, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, + 0x61, 0x64, 0x12, 0x0e, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, + 0x49, 0x44, 0x1a, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, + 0x22, 0x00, 0x12, 0x30, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, + 0x64, 0x73, 0x12, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, + 0x49, 0x44, 0x73, 0x1a, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, + 0x64, 0x73, 0x22, 0x00, 0x12, 0x3b, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, + 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x12, 0x18, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, + 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x00, 0x30, + 0x01, 0x12, 0x38, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x57, 0x6f, 0x72, + 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x12, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4e, + 0x6f, 0x64, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x0d, 0x2e, 0x70, 0x62, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x22, 0x00, 0x12, 0x3c, 0x0a, 0x12, 0x47, + 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x12, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, + 0x44, 0x73, 0x1a, 0x13, 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, + 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x4a, 0x0a, 0x12, 0x53, 0x65, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x1d, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, + 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x13, + 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x14, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, + 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x1f, 0x2e, + 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1f, + 0x2e, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 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, 0x3b, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x12, + 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x14, 0x2e, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, + 0x6d, 0x61, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, + 0x42, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, + 0x64, 0x12, 0x11, 0x2e, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, + 0x00, 0x30, 0x01, 0x12, 0x45, 0x0a, 0x0f, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x57, 0x6f, + 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x70, 0x6c, + 0x61, 0x63, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1a, 0x2e, 0x70, 0x62, 0x2e, + 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x4a, 0x0a, 0x0e, 0x52, 0x65, + 0x6d, 0x6f, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x19, 0x2e, 0x70, + 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, + 0x6f, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x56, 0x0a, 0x12, 0x44, 0x69, 0x73, 0x73, 0x6f, 0x63, + 0x69, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1d, 0x2e, 0x70, + 0x62, 0x2e, 0x44, 0x69, 0x73, 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, + 0x6c, 0x6f, 0x61, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1d, 0x2e, 0x70, 0x62, + 0x2e, 0x44, 0x69, 0x73, 0x73, 0x6f, 0x63, 0x69, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, + 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x4d, + 0x0a, 0x0f, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, + 0x64, 0x12, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x57, 0x6f, + 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x1a, 0x2e, + 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, + 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x4e, 0x0a, + 0x0f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, + 0x12, 0x1a, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x57, 0x6f, 0x72, + 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x19, 0x2e, 0x70, + 0x62, 0x2e, 0x41, 0x74, 0x74, 0x61, 0x63, 0x68, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, + 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, + 0x44, 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, 0x19, 0x2e, 0x70, 0x62, 0x2e, 0x41, 0x74, 0x74, 0x61, 0x63, 0x68, + 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, + 0x00, 0x28, 0x01, 0x30, 0x01, 0x42, 0x28, 0x5a, 0x26, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, + 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x65, 0x72, 0x75, 0x32, 0x2f, + 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x67, 0x65, 0x6e, 0x3b, 0x70, 0x62, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -7453,7 +7743,7 @@ func file_rpc_gen_core_proto_rawDescGZIP() []byte { } var file_rpc_gen_core_proto_enumTypes = make([]protoimpl.EnumInfo, 4) -var file_rpc_gen_core_proto_msgTypes = make([]protoimpl.MessageInfo, 134) +var file_rpc_gen_core_proto_msgTypes = make([]protoimpl.MessageInfo, 140) var file_rpc_gen_core_proto_goTypes = []interface{}{ (TriOpt)(0), // 0: pb.TriOpt (StdStreamType)(0), // 1: pb.StdStreamType @@ -7467,321 +7757,336 @@ var file_rpc_gen_core_proto_goTypes = []interface{}{ (*Pod)(nil), // 9: pb.Pod (*Pods)(nil), // 10: pb.Pods (*PodResource)(nil), // 11: pb.PodResource - (*NodeResource)(nil), // 12: pb.NodeResource - (*ListNetworkOptions)(nil), // 13: pb.ListNetworkOptions - (*ConnectNetworkOptions)(nil), // 14: pb.ConnectNetworkOptions - (*DisconnectNetworkOptions)(nil), // 15: pb.DisconnectNetworkOptions - (*Network)(nil), // 16: pb.Network - (*Networks)(nil), // 17: pb.Networks - (*Node)(nil), // 18: pb.Node - (*Nodes)(nil), // 19: pb.Nodes - (*NodeAvailable)(nil), // 20: pb.NodeAvailable - (*SetNodeOptions)(nil), // 21: pb.SetNodeOptions - (*SetNodeStatusOptions)(nil), // 22: pb.SetNodeStatusOptions - (*GetNodeStatusOptions)(nil), // 23: pb.GetNodeStatusOptions - (*NodeStatusStreamMessage)(nil), // 24: pb.NodeStatusStreamMessage - (*NodeFilter)(nil), // 25: pb.NodeFilter - (*Workload)(nil), // 26: pb.Workload - (*WorkloadStatus)(nil), // 27: pb.WorkloadStatus - (*WorkloadsStatus)(nil), // 28: pb.WorkloadsStatus - (*SetWorkloadsStatusOptions)(nil), // 29: pb.SetWorkloadsStatusOptions - (*WorkloadStatusStreamOptions)(nil), // 30: pb.WorkloadStatusStreamOptions - (*WorkloadStatusStreamMessage)(nil), // 31: pb.WorkloadStatusStreamMessage - (*Workloads)(nil), // 32: pb.Workloads - (*WorkloadID)(nil), // 33: pb.WorkloadID - (*WorkloadIDs)(nil), // 34: pb.WorkloadIDs - (*RemoveWorkloadOptions)(nil), // 35: pb.RemoveWorkloadOptions - (*DissociateWorkloadOptions)(nil), // 36: pb.DissociateWorkloadOptions - (*ReallocOptions)(nil), // 37: pb.ReallocOptions - (*AddPodOptions)(nil), // 38: pb.AddPodOptions - (*RemovePodOptions)(nil), // 39: pb.RemovePodOptions - (*GetPodOptions)(nil), // 40: pb.GetPodOptions - (*AddNodeOptions)(nil), // 41: pb.AddNodeOptions - (*RemoveNodeOptions)(nil), // 42: pb.RemoveNodeOptions - (*GetNodeOptions)(nil), // 43: pb.GetNodeOptions - (*GetNodeResourceOptions)(nil), // 44: pb.GetNodeResourceOptions - (*ListNodesOptions)(nil), // 45: pb.ListNodesOptions - (*Build)(nil), // 46: pb.Build - (*Builds)(nil), // 47: pb.Builds - (*BuildImageOptions)(nil), // 48: pb.BuildImageOptions - (*HookOptions)(nil), // 49: pb.HookOptions - (*HealthCheckOptions)(nil), // 50: pb.HealthCheckOptions - (*LogOptions)(nil), // 51: pb.LogOptions - (*EntrypointOptions)(nil), // 52: pb.EntrypointOptions - (*ResourceOptions)(nil), // 53: pb.ResourceOptions - (*Resource)(nil), // 54: pb.Resource - (*Volume)(nil), // 55: pb.Volume - (*DeployOptions)(nil), // 56: pb.DeployOptions - (*ReplaceOptions)(nil), // 57: pb.ReplaceOptions - (*CacheImageOptions)(nil), // 58: pb.CacheImageOptions - (*RemoveImageOptions)(nil), // 59: pb.RemoveImageOptions - (*ListImageOptions)(nil), // 60: pb.ListImageOptions - (*CopyPaths)(nil), // 61: pb.CopyPaths - (*CopyOptions)(nil), // 62: pb.CopyOptions - (*FileOwner)(nil), // 63: pb.FileOwner - (*FileMode)(nil), // 64: pb.FileMode - (*SendOptions)(nil), // 65: pb.SendOptions - (*ErrorDetail)(nil), // 66: pb.ErrorDetail - (*BuildImageMessage)(nil), // 67: pb.BuildImageMessage - (*CreateWorkloadMessage)(nil), // 68: pb.CreateWorkloadMessage - (*ReplaceWorkloadMessage)(nil), // 69: pb.ReplaceWorkloadMessage - (*CacheImageMessage)(nil), // 70: pb.CacheImageMessage - (*RemoveImageMessage)(nil), // 71: pb.RemoveImageMessage - (*ImageItem)(nil), // 72: pb.ImageItem - (*ListImageMessage)(nil), // 73: pb.ListImageMessage - (*RemoveWorkloadMessage)(nil), // 74: pb.RemoveWorkloadMessage - (*DissociateWorkloadMessage)(nil), // 75: pb.DissociateWorkloadMessage - (*ReallocResourceMessage)(nil), // 76: pb.ReallocResourceMessage - (*CopyMessage)(nil), // 77: pb.CopyMessage - (*SendMessage)(nil), // 78: pb.SendMessage - (*AttachWorkloadMessage)(nil), // 79: pb.AttachWorkloadMessage - (*RunAndWaitOptions)(nil), // 80: pb.RunAndWaitOptions - (*ControlWorkloadOptions)(nil), // 81: pb.ControlWorkloadOptions - (*ControlWorkloadMessage)(nil), // 82: pb.ControlWorkloadMessage - (*LogStreamOptions)(nil), // 83: pb.LogStreamOptions - (*LogStreamMessage)(nil), // 84: pb.LogStreamMessage - (*ExecuteWorkloadOptions)(nil), // 85: pb.ExecuteWorkloadOptions - (*CapacityMessage)(nil), // 86: pb.CapacityMessage - nil, // 87: pb.ListWorkloadsOptions.LabelsEntry - nil, // 88: pb.Node.CpuEntry - nil, // 89: pb.Node.LabelsEntry - nil, // 90: pb.Node.InitCpuEntry - nil, // 91: pb.Node.NumaEntry - nil, // 92: pb.Node.NumaMemoryEntry - nil, // 93: pb.Node.InitVolumeEntry - nil, // 94: pb.Node.VolumeEntry - nil, // 95: pb.Node.InitNumaMemoryEntry - nil, // 96: pb.SetNodeOptions.DeltaCpuEntry - nil, // 97: pb.SetNodeOptions.DeltaNumaMemoryEntry - nil, // 98: pb.SetNodeOptions.NumaEntry - nil, // 99: pb.SetNodeOptions.LabelsEntry - nil, // 100: pb.SetNodeOptions.DeltaVolumeEntry - nil, // 101: pb.NodeFilter.LabelsEntry - nil, // 102: pb.Workload.LabelsEntry - nil, // 103: pb.Workload.PublishEntry - nil, // 104: pb.WorkloadStatus.NetworksEntry - nil, // 105: pb.WorkloadStatusStreamOptions.LabelsEntry - nil, // 106: pb.AddNodeOptions.LabelsEntry - nil, // 107: pb.AddNodeOptions.NumaEntry - nil, // 108: pb.AddNodeOptions.NumaMemoryEntry - nil, // 109: pb.AddNodeOptions.VolumeMapEntry - nil, // 110: pb.GetNodeOptions.LabelsEntry - nil, // 111: pb.ListNodesOptions.LabelsEntry - nil, // 112: pb.Build.EnvsEntry - nil, // 113: pb.Build.ArgsEntry - nil, // 114: pb.Build.LabelsEntry - nil, // 115: pb.Build.ArtifactsEntry - nil, // 116: pb.Build.CacheEntry - nil, // 117: pb.Builds.BuildsEntry - nil, // 118: pb.LogOptions.ConfigEntry - nil, // 119: pb.EntrypointOptions.SysctlsEntry - nil, // 120: pb.Resource.CpuEntry - nil, // 121: pb.Resource.VolumePlanLimitEntry - nil, // 122: pb.Resource.VolumePlanRequestEntry - nil, // 123: pb.Volume.VolumeEntry - nil, // 124: pb.DeployOptions.NetworksEntry - nil, // 125: pb.DeployOptions.LabelsEntry - nil, // 126: pb.DeployOptions.NodelabelsEntry - nil, // 127: pb.DeployOptions.DataEntry - nil, // 128: pb.DeployOptions.ModesEntry - nil, // 129: pb.DeployOptions.OwnersEntry - nil, // 130: pb.ReplaceOptions.FilterLabelsEntry - nil, // 131: pb.ReplaceOptions.CopyEntry - nil, // 132: pb.CopyOptions.TargetsEntry - nil, // 133: pb.SendOptions.DataEntry - nil, // 134: pb.SendOptions.ModesEntry - nil, // 135: pb.SendOptions.OwnersEntry - nil, // 136: pb.CreateWorkloadMessage.PublishEntry - nil, // 137: pb.CapacityMessage.NodeCapacitiesEntry + (*StringSlice)(nil), // 12: pb.StringSlice + (*RawParam)(nil), // 13: pb.RawParam + (*NodeResource)(nil), // 14: pb.NodeResource + (*ListNetworkOptions)(nil), // 15: pb.ListNetworkOptions + (*ConnectNetworkOptions)(nil), // 16: pb.ConnectNetworkOptions + (*DisconnectNetworkOptions)(nil), // 17: pb.DisconnectNetworkOptions + (*Network)(nil), // 18: pb.Network + (*Networks)(nil), // 19: pb.Networks + (*Node)(nil), // 20: pb.Node + (*Nodes)(nil), // 21: pb.Nodes + (*NodeAvailable)(nil), // 22: pb.NodeAvailable + (*SetNodeOptions)(nil), // 23: pb.SetNodeOptions + (*SetNodeStatusOptions)(nil), // 24: pb.SetNodeStatusOptions + (*GetNodeStatusOptions)(nil), // 25: pb.GetNodeStatusOptions + (*NodeStatusStreamMessage)(nil), // 26: pb.NodeStatusStreamMessage + (*NodeFilter)(nil), // 27: pb.NodeFilter + (*Workload)(nil), // 28: pb.Workload + (*WorkloadStatus)(nil), // 29: pb.WorkloadStatus + (*WorkloadsStatus)(nil), // 30: pb.WorkloadsStatus + (*SetWorkloadsStatusOptions)(nil), // 31: pb.SetWorkloadsStatusOptions + (*WorkloadStatusStreamOptions)(nil), // 32: pb.WorkloadStatusStreamOptions + (*WorkloadStatusStreamMessage)(nil), // 33: pb.WorkloadStatusStreamMessage + (*Workloads)(nil), // 34: pb.Workloads + (*WorkloadID)(nil), // 35: pb.WorkloadID + (*WorkloadIDs)(nil), // 36: pb.WorkloadIDs + (*RemoveWorkloadOptions)(nil), // 37: pb.RemoveWorkloadOptions + (*DissociateWorkloadOptions)(nil), // 38: pb.DissociateWorkloadOptions + (*ReallocOptions)(nil), // 39: pb.ReallocOptions + (*AddPodOptions)(nil), // 40: pb.AddPodOptions + (*RemovePodOptions)(nil), // 41: pb.RemovePodOptions + (*GetPodOptions)(nil), // 42: pb.GetPodOptions + (*AddNodeOptions)(nil), // 43: pb.AddNodeOptions + (*RemoveNodeOptions)(nil), // 44: pb.RemoveNodeOptions + (*GetNodeOptions)(nil), // 45: pb.GetNodeOptions + (*GetNodeResourceOptions)(nil), // 46: pb.GetNodeResourceOptions + (*ListNodesOptions)(nil), // 47: pb.ListNodesOptions + (*Build)(nil), // 48: pb.Build + (*Builds)(nil), // 49: pb.Builds + (*BuildImageOptions)(nil), // 50: pb.BuildImageOptions + (*HookOptions)(nil), // 51: pb.HookOptions + (*HealthCheckOptions)(nil), // 52: pb.HealthCheckOptions + (*LogOptions)(nil), // 53: pb.LogOptions + (*EntrypointOptions)(nil), // 54: pb.EntrypointOptions + (*ResourceOptions)(nil), // 55: pb.ResourceOptions + (*Resource)(nil), // 56: pb.Resource + (*Volume)(nil), // 57: pb.Volume + (*DeployOptions)(nil), // 58: pb.DeployOptions + (*ReplaceOptions)(nil), // 59: pb.ReplaceOptions + (*CacheImageOptions)(nil), // 60: pb.CacheImageOptions + (*RemoveImageOptions)(nil), // 61: pb.RemoveImageOptions + (*ListImageOptions)(nil), // 62: pb.ListImageOptions + (*CopyPaths)(nil), // 63: pb.CopyPaths + (*CopyOptions)(nil), // 64: pb.CopyOptions + (*FileOwner)(nil), // 65: pb.FileOwner + (*FileMode)(nil), // 66: pb.FileMode + (*SendOptions)(nil), // 67: pb.SendOptions + (*ErrorDetail)(nil), // 68: pb.ErrorDetail + (*BuildImageMessage)(nil), // 69: pb.BuildImageMessage + (*CreateWorkloadMessage)(nil), // 70: pb.CreateWorkloadMessage + (*ReplaceWorkloadMessage)(nil), // 71: pb.ReplaceWorkloadMessage + (*CacheImageMessage)(nil), // 72: pb.CacheImageMessage + (*RemoveImageMessage)(nil), // 73: pb.RemoveImageMessage + (*ImageItem)(nil), // 74: pb.ImageItem + (*ListImageMessage)(nil), // 75: pb.ListImageMessage + (*RemoveWorkloadMessage)(nil), // 76: pb.RemoveWorkloadMessage + (*DissociateWorkloadMessage)(nil), // 77: pb.DissociateWorkloadMessage + (*ReallocResourceMessage)(nil), // 78: pb.ReallocResourceMessage + (*CopyMessage)(nil), // 79: pb.CopyMessage + (*SendMessage)(nil), // 80: pb.SendMessage + (*AttachWorkloadMessage)(nil), // 81: pb.AttachWorkloadMessage + (*RunAndWaitOptions)(nil), // 82: pb.RunAndWaitOptions + (*ControlWorkloadOptions)(nil), // 83: pb.ControlWorkloadOptions + (*ControlWorkloadMessage)(nil), // 84: pb.ControlWorkloadMessage + (*LogStreamOptions)(nil), // 85: pb.LogStreamOptions + (*LogStreamMessage)(nil), // 86: pb.LogStreamMessage + (*ExecuteWorkloadOptions)(nil), // 87: pb.ExecuteWorkloadOptions + (*CapacityMessage)(nil), // 88: pb.CapacityMessage + nil, // 89: pb.ListWorkloadsOptions.LabelsEntry + nil, // 90: pb.Node.CpuEntry + nil, // 91: pb.Node.LabelsEntry + nil, // 92: pb.Node.InitCpuEntry + nil, // 93: pb.Node.NumaEntry + nil, // 94: pb.Node.NumaMemoryEntry + nil, // 95: pb.Node.InitVolumeEntry + nil, // 96: pb.Node.VolumeEntry + nil, // 97: pb.Node.InitNumaMemoryEntry + nil, // 98: pb.SetNodeOptions.DeltaCpuEntry + nil, // 99: pb.SetNodeOptions.DeltaNumaMemoryEntry + nil, // 100: pb.SetNodeOptions.NumaEntry + nil, // 101: pb.SetNodeOptions.LabelsEntry + nil, // 102: pb.SetNodeOptions.DeltaVolumeEntry + nil, // 103: pb.SetNodeOptions.ResourceOptsEntry + nil, // 104: pb.NodeFilter.LabelsEntry + nil, // 105: pb.Workload.LabelsEntry + nil, // 106: pb.Workload.PublishEntry + nil, // 107: pb.WorkloadStatus.NetworksEntry + nil, // 108: pb.WorkloadStatusStreamOptions.LabelsEntry + nil, // 109: pb.ReallocOptions.ResourceOptsEntry + nil, // 110: pb.AddNodeOptions.LabelsEntry + nil, // 111: pb.AddNodeOptions.NumaEntry + nil, // 112: pb.AddNodeOptions.NumaMemoryEntry + nil, // 113: pb.AddNodeOptions.VolumeMapEntry + nil, // 114: pb.AddNodeOptions.ResourceOptsEntry + nil, // 115: pb.GetNodeOptions.LabelsEntry + nil, // 116: pb.ListNodesOptions.LabelsEntry + nil, // 117: pb.Build.EnvsEntry + nil, // 118: pb.Build.ArgsEntry + nil, // 119: pb.Build.LabelsEntry + nil, // 120: pb.Build.ArtifactsEntry + nil, // 121: pb.Build.CacheEntry + nil, // 122: pb.Builds.BuildsEntry + nil, // 123: pb.LogOptions.ConfigEntry + nil, // 124: pb.EntrypointOptions.SysctlsEntry + nil, // 125: pb.Resource.CpuEntry + nil, // 126: pb.Resource.VolumePlanLimitEntry + nil, // 127: pb.Resource.VolumePlanRequestEntry + nil, // 128: pb.Volume.VolumeEntry + nil, // 129: pb.DeployOptions.NetworksEntry + nil, // 130: pb.DeployOptions.LabelsEntry + nil, // 131: pb.DeployOptions.NodelabelsEntry + nil, // 132: pb.DeployOptions.DataEntry + nil, // 133: pb.DeployOptions.ModesEntry + nil, // 134: pb.DeployOptions.OwnersEntry + nil, // 135: pb.DeployOptions.ResourceOptsEntry + nil, // 136: pb.ReplaceOptions.FilterLabelsEntry + nil, // 137: pb.ReplaceOptions.CopyEntry + nil, // 138: pb.CopyOptions.TargetsEntry + nil, // 139: pb.SendOptions.DataEntry + nil, // 140: pb.SendOptions.ModesEntry + nil, // 141: pb.SendOptions.OwnersEntry + nil, // 142: pb.CreateWorkloadMessage.PublishEntry + nil, // 143: pb.CapacityMessage.NodeCapacitiesEntry } var file_rpc_gen_core_proto_depIdxs = []int32{ - 87, // 0: pb.ListWorkloadsOptions.labels:type_name -> pb.ListWorkloadsOptions.LabelsEntry + 89, // 0: pb.ListWorkloadsOptions.labels:type_name -> pb.ListWorkloadsOptions.LabelsEntry 9, // 1: pb.Pods.pods:type_name -> pb.Pod - 12, // 2: pb.PodResource.nodes_resource:type_name -> pb.NodeResource - 16, // 3: pb.Networks.networks:type_name -> pb.Network - 88, // 4: pb.Node.cpu:type_name -> pb.Node.CpuEntry - 89, // 5: pb.Node.labels:type_name -> pb.Node.LabelsEntry - 90, // 6: pb.Node.init_cpu:type_name -> pb.Node.InitCpuEntry - 91, // 7: pb.Node.numa:type_name -> pb.Node.NumaEntry - 92, // 8: pb.Node.numa_memory:type_name -> pb.Node.NumaMemoryEntry - 93, // 9: pb.Node.init_volume:type_name -> pb.Node.InitVolumeEntry - 94, // 10: pb.Node.volume:type_name -> pb.Node.VolumeEntry - 95, // 11: pb.Node.init_numa_memory:type_name -> pb.Node.InitNumaMemoryEntry - 18, // 12: pb.Nodes.nodes:type_name -> pb.Node - 96, // 13: pb.SetNodeOptions.delta_cpu:type_name -> pb.SetNodeOptions.DeltaCpuEntry - 97, // 14: pb.SetNodeOptions.delta_numa_memory:type_name -> pb.SetNodeOptions.DeltaNumaMemoryEntry - 98, // 15: pb.SetNodeOptions.numa:type_name -> pb.SetNodeOptions.NumaEntry - 99, // 16: pb.SetNodeOptions.labels:type_name -> pb.SetNodeOptions.LabelsEntry - 100, // 17: pb.SetNodeOptions.delta_volume:type_name -> pb.SetNodeOptions.DeltaVolumeEntry - 0, // 18: pb.SetNodeOptions.bypass_opt:type_name -> pb.TriOpt - 101, // 19: pb.NodeFilter.labels:type_name -> pb.NodeFilter.LabelsEntry - 102, // 20: pb.Workload.labels:type_name -> pb.Workload.LabelsEntry - 103, // 21: pb.Workload.publish:type_name -> pb.Workload.PublishEntry - 27, // 22: pb.Workload.status:type_name -> pb.WorkloadStatus - 54, // 23: pb.Workload.resource:type_name -> pb.Resource - 104, // 24: pb.WorkloadStatus.networks:type_name -> pb.WorkloadStatus.NetworksEntry - 27, // 25: pb.WorkloadsStatus.status:type_name -> pb.WorkloadStatus - 27, // 26: pb.SetWorkloadsStatusOptions.status:type_name -> pb.WorkloadStatus - 105, // 27: pb.WorkloadStatusStreamOptions.labels:type_name -> pb.WorkloadStatusStreamOptions.LabelsEntry - 26, // 28: pb.WorkloadStatusStreamMessage.workload:type_name -> pb.Workload - 27, // 29: pb.WorkloadStatusStreamMessage.status:type_name -> pb.WorkloadStatus - 26, // 30: pb.Workloads.workloads:type_name -> pb.Workload - 0, // 31: pb.ReallocOptions.bind_cpu_opt:type_name -> pb.TriOpt - 53, // 32: pb.ReallocOptions.resource_opts:type_name -> pb.ResourceOptions - 106, // 33: pb.AddNodeOptions.labels:type_name -> pb.AddNodeOptions.LabelsEntry - 107, // 34: pb.AddNodeOptions.numa:type_name -> pb.AddNodeOptions.NumaEntry - 108, // 35: pb.AddNodeOptions.numa_memory:type_name -> pb.AddNodeOptions.NumaMemoryEntry - 109, // 36: pb.AddNodeOptions.volume_map:type_name -> pb.AddNodeOptions.VolumeMapEntry - 110, // 37: pb.GetNodeOptions.labels:type_name -> pb.GetNodeOptions.LabelsEntry - 43, // 38: pb.GetNodeResourceOptions.opts:type_name -> pb.GetNodeOptions - 111, // 39: pb.ListNodesOptions.labels:type_name -> pb.ListNodesOptions.LabelsEntry - 112, // 40: pb.Build.envs:type_name -> pb.Build.EnvsEntry - 113, // 41: pb.Build.args:type_name -> pb.Build.ArgsEntry - 114, // 42: pb.Build.labels:type_name -> pb.Build.LabelsEntry - 115, // 43: pb.Build.artifacts:type_name -> pb.Build.ArtifactsEntry - 116, // 44: pb.Build.cache:type_name -> pb.Build.CacheEntry - 117, // 45: pb.Builds.builds:type_name -> pb.Builds.BuildsEntry - 47, // 46: pb.BuildImageOptions.builds:type_name -> pb.Builds - 2, // 47: pb.BuildImageOptions.build_method:type_name -> pb.BuildImageOptions.BuildMethod - 118, // 48: pb.LogOptions.config:type_name -> pb.LogOptions.ConfigEntry - 51, // 49: pb.EntrypointOptions.log:type_name -> pb.LogOptions - 50, // 50: pb.EntrypointOptions.healthcheck:type_name -> pb.HealthCheckOptions - 49, // 51: pb.EntrypointOptions.hook:type_name -> pb.HookOptions - 119, // 52: pb.EntrypointOptions.sysctls:type_name -> pb.EntrypointOptions.SysctlsEntry - 120, // 53: pb.Resource.cpu:type_name -> pb.Resource.CpuEntry - 121, // 54: pb.Resource.volume_plan_limit:type_name -> pb.Resource.VolumePlanLimitEntry - 122, // 55: pb.Resource.volume_plan_request:type_name -> pb.Resource.VolumePlanRequestEntry - 123, // 56: pb.Volume.volume:type_name -> pb.Volume.VolumeEntry - 52, // 57: pb.DeployOptions.entrypoint:type_name -> pb.EntrypointOptions - 124, // 58: pb.DeployOptions.networks:type_name -> pb.DeployOptions.NetworksEntry - 125, // 59: pb.DeployOptions.labels:type_name -> pb.DeployOptions.LabelsEntry - 126, // 60: pb.DeployOptions.nodelabels:type_name -> pb.DeployOptions.NodelabelsEntry - 3, // 61: pb.DeployOptions.deploy_strategy:type_name -> pb.DeployOptions.Strategy - 127, // 62: pb.DeployOptions.data:type_name -> pb.DeployOptions.DataEntry - 53, // 63: pb.DeployOptions.resource_opts:type_name -> pb.ResourceOptions - 25, // 64: pb.DeployOptions.node_filter:type_name -> pb.NodeFilter - 128, // 65: pb.DeployOptions.modes:type_name -> pb.DeployOptions.ModesEntry - 129, // 66: pb.DeployOptions.owners:type_name -> pb.DeployOptions.OwnersEntry - 56, // 67: pb.ReplaceOptions.deployOpt:type_name -> pb.DeployOptions - 130, // 68: pb.ReplaceOptions.filter_labels:type_name -> pb.ReplaceOptions.FilterLabelsEntry - 131, // 69: pb.ReplaceOptions.copy:type_name -> pb.ReplaceOptions.CopyEntry - 132, // 70: pb.CopyOptions.targets:type_name -> pb.CopyOptions.TargetsEntry - 133, // 71: pb.SendOptions.data:type_name -> pb.SendOptions.DataEntry - 134, // 72: pb.SendOptions.modes:type_name -> pb.SendOptions.ModesEntry - 135, // 73: pb.SendOptions.owners:type_name -> pb.SendOptions.OwnersEntry - 66, // 74: pb.BuildImageMessage.error_detail:type_name -> pb.ErrorDetail - 136, // 75: pb.CreateWorkloadMessage.publish:type_name -> pb.CreateWorkloadMessage.PublishEntry - 54, // 76: pb.CreateWorkloadMessage.resource:type_name -> pb.Resource - 68, // 77: pb.ReplaceWorkloadMessage.create:type_name -> pb.CreateWorkloadMessage - 74, // 78: pb.ReplaceWorkloadMessage.remove:type_name -> pb.RemoveWorkloadMessage - 72, // 79: pb.ListImageMessage.images:type_name -> pb.ImageItem - 1, // 80: pb.AttachWorkloadMessage.std_stream_type:type_name -> pb.StdStreamType - 56, // 81: pb.RunAndWaitOptions.deploy_options:type_name -> pb.DeployOptions - 1, // 82: pb.LogStreamMessage.std_stream_type:type_name -> pb.StdStreamType - 137, // 83: pb.CapacityMessage.node_capacities:type_name -> pb.CapacityMessage.NodeCapacitiesEntry - 46, // 84: pb.Builds.BuildsEntry.value:type_name -> pb.Build - 55, // 85: pb.Resource.VolumePlanLimitEntry.value:type_name -> pb.Volume - 55, // 86: pb.Resource.VolumePlanRequestEntry.value:type_name -> pb.Volume - 64, // 87: pb.DeployOptions.ModesEntry.value:type_name -> pb.FileMode - 63, // 88: pb.DeployOptions.OwnersEntry.value:type_name -> pb.FileOwner - 61, // 89: pb.CopyOptions.TargetsEntry.value:type_name -> pb.CopyPaths - 64, // 90: pb.SendOptions.ModesEntry.value:type_name -> pb.FileMode - 63, // 91: pb.SendOptions.OwnersEntry.value:type_name -> pb.FileOwner - 4, // 92: pb.CoreRPC.Info:input_type -> pb.Empty - 4, // 93: pb.CoreRPC.WatchServiceStatus:input_type -> pb.Empty - 13, // 94: pb.CoreRPC.ListNetworks:input_type -> pb.ListNetworkOptions - 14, // 95: pb.CoreRPC.ConnectNetwork:input_type -> pb.ConnectNetworkOptions - 15, // 96: pb.CoreRPC.DisconnectNetwork:input_type -> pb.DisconnectNetworkOptions - 38, // 97: pb.CoreRPC.AddPod:input_type -> pb.AddPodOptions - 39, // 98: pb.CoreRPC.RemovePod:input_type -> pb.RemovePodOptions - 40, // 99: pb.CoreRPC.GetPod:input_type -> pb.GetPodOptions - 4, // 100: pb.CoreRPC.ListPods:input_type -> pb.Empty - 40, // 101: pb.CoreRPC.GetPodResource:input_type -> pb.GetPodOptions - 40, // 102: pb.CoreRPC.PodResourceStream:input_type -> pb.GetPodOptions - 41, // 103: pb.CoreRPC.AddNode:input_type -> pb.AddNodeOptions - 42, // 104: pb.CoreRPC.RemoveNode:input_type -> pb.RemoveNodeOptions - 45, // 105: pb.CoreRPC.ListPodNodes:input_type -> pb.ListNodesOptions - 45, // 106: pb.CoreRPC.PodNodesStream:input_type -> pb.ListNodesOptions - 43, // 107: pb.CoreRPC.GetNode:input_type -> pb.GetNodeOptions - 43, // 108: pb.CoreRPC.GetNodeEngine:input_type -> pb.GetNodeOptions - 21, // 109: pb.CoreRPC.SetNode:input_type -> pb.SetNodeOptions - 22, // 110: pb.CoreRPC.SetNodeStatus:input_type -> pb.SetNodeStatusOptions - 23, // 111: pb.CoreRPC.GetNodeStatus:input_type -> pb.GetNodeStatusOptions - 4, // 112: pb.CoreRPC.NodeStatusStream:input_type -> pb.Empty - 44, // 113: pb.CoreRPC.GetNodeResource:input_type -> pb.GetNodeResourceOptions - 56, // 114: pb.CoreRPC.CalculateCapacity:input_type -> pb.DeployOptions - 33, // 115: pb.CoreRPC.GetWorkload:input_type -> pb.WorkloadID - 34, // 116: pb.CoreRPC.GetWorkloads:input_type -> pb.WorkloadIDs - 8, // 117: pb.CoreRPC.ListWorkloads:input_type -> pb.ListWorkloadsOptions - 43, // 118: pb.CoreRPC.ListNodeWorkloads:input_type -> pb.GetNodeOptions - 34, // 119: pb.CoreRPC.GetWorkloadsStatus:input_type -> pb.WorkloadIDs - 29, // 120: pb.CoreRPC.SetWorkloadsStatus:input_type -> pb.SetWorkloadsStatusOptions - 30, // 121: pb.CoreRPC.WorkloadStatusStream:input_type -> pb.WorkloadStatusStreamOptions - 62, // 122: pb.CoreRPC.Copy:input_type -> pb.CopyOptions - 65, // 123: pb.CoreRPC.Send:input_type -> pb.SendOptions - 48, // 124: pb.CoreRPC.BuildImage:input_type -> pb.BuildImageOptions - 58, // 125: pb.CoreRPC.CacheImage:input_type -> pb.CacheImageOptions - 59, // 126: pb.CoreRPC.RemoveImage:input_type -> pb.RemoveImageOptions - 60, // 127: pb.CoreRPC.ListImage:input_type -> pb.ListImageOptions - 56, // 128: pb.CoreRPC.CreateWorkload:input_type -> pb.DeployOptions - 57, // 129: pb.CoreRPC.ReplaceWorkload:input_type -> pb.ReplaceOptions - 35, // 130: pb.CoreRPC.RemoveWorkload:input_type -> pb.RemoveWorkloadOptions - 36, // 131: pb.CoreRPC.DissociateWorkload:input_type -> pb.DissociateWorkloadOptions - 81, // 132: pb.CoreRPC.ControlWorkload:input_type -> pb.ControlWorkloadOptions - 85, // 133: pb.CoreRPC.ExecuteWorkload:input_type -> pb.ExecuteWorkloadOptions - 37, // 134: pb.CoreRPC.ReallocResource:input_type -> pb.ReallocOptions - 83, // 135: pb.CoreRPC.LogStream:input_type -> pb.LogStreamOptions - 80, // 136: pb.CoreRPC.RunAndWait:input_type -> pb.RunAndWaitOptions - 5, // 137: pb.CoreRPC.Info:output_type -> pb.CoreInfo - 6, // 138: pb.CoreRPC.WatchServiceStatus:output_type -> pb.ServiceStatus - 17, // 139: pb.CoreRPC.ListNetworks:output_type -> pb.Networks - 16, // 140: pb.CoreRPC.ConnectNetwork:output_type -> pb.Network - 4, // 141: pb.CoreRPC.DisconnectNetwork:output_type -> pb.Empty - 9, // 142: pb.CoreRPC.AddPod:output_type -> pb.Pod - 4, // 143: pb.CoreRPC.RemovePod:output_type -> pb.Empty - 9, // 144: pb.CoreRPC.GetPod:output_type -> pb.Pod - 10, // 145: pb.CoreRPC.ListPods:output_type -> pb.Pods - 11, // 146: pb.CoreRPC.GetPodResource:output_type -> pb.PodResource - 12, // 147: pb.CoreRPC.PodResourceStream:output_type -> pb.NodeResource - 18, // 148: pb.CoreRPC.AddNode:output_type -> pb.Node - 4, // 149: pb.CoreRPC.RemoveNode:output_type -> pb.Empty - 19, // 150: pb.CoreRPC.ListPodNodes:output_type -> pb.Nodes - 18, // 151: pb.CoreRPC.PodNodesStream:output_type -> pb.Node - 18, // 152: pb.CoreRPC.GetNode:output_type -> pb.Node - 7, // 153: pb.CoreRPC.GetNodeEngine:output_type -> pb.Engine - 18, // 154: pb.CoreRPC.SetNode:output_type -> pb.Node - 4, // 155: pb.CoreRPC.SetNodeStatus:output_type -> pb.Empty - 24, // 156: pb.CoreRPC.GetNodeStatus:output_type -> pb.NodeStatusStreamMessage - 24, // 157: pb.CoreRPC.NodeStatusStream:output_type -> pb.NodeStatusStreamMessage - 12, // 158: pb.CoreRPC.GetNodeResource:output_type -> pb.NodeResource - 86, // 159: pb.CoreRPC.CalculateCapacity:output_type -> pb.CapacityMessage - 26, // 160: pb.CoreRPC.GetWorkload:output_type -> pb.Workload - 32, // 161: pb.CoreRPC.GetWorkloads:output_type -> pb.Workloads - 26, // 162: pb.CoreRPC.ListWorkloads:output_type -> pb.Workload - 32, // 163: pb.CoreRPC.ListNodeWorkloads:output_type -> pb.Workloads - 28, // 164: pb.CoreRPC.GetWorkloadsStatus:output_type -> pb.WorkloadsStatus - 28, // 165: pb.CoreRPC.SetWorkloadsStatus:output_type -> pb.WorkloadsStatus - 31, // 166: pb.CoreRPC.WorkloadStatusStream:output_type -> pb.WorkloadStatusStreamMessage - 77, // 167: pb.CoreRPC.Copy:output_type -> pb.CopyMessage - 78, // 168: pb.CoreRPC.Send:output_type -> pb.SendMessage - 67, // 169: pb.CoreRPC.BuildImage:output_type -> pb.BuildImageMessage - 70, // 170: pb.CoreRPC.CacheImage:output_type -> pb.CacheImageMessage - 71, // 171: pb.CoreRPC.RemoveImage:output_type -> pb.RemoveImageMessage - 73, // 172: pb.CoreRPC.ListImage:output_type -> pb.ListImageMessage - 68, // 173: pb.CoreRPC.CreateWorkload:output_type -> pb.CreateWorkloadMessage - 69, // 174: pb.CoreRPC.ReplaceWorkload:output_type -> pb.ReplaceWorkloadMessage - 74, // 175: pb.CoreRPC.RemoveWorkload:output_type -> pb.RemoveWorkloadMessage - 75, // 176: pb.CoreRPC.DissociateWorkload:output_type -> pb.DissociateWorkloadMessage - 82, // 177: pb.CoreRPC.ControlWorkload:output_type -> pb.ControlWorkloadMessage - 79, // 178: pb.CoreRPC.ExecuteWorkload:output_type -> pb.AttachWorkloadMessage - 76, // 179: pb.CoreRPC.ReallocResource:output_type -> pb.ReallocResourceMessage - 84, // 180: pb.CoreRPC.LogStream:output_type -> pb.LogStreamMessage - 79, // 181: pb.CoreRPC.RunAndWait:output_type -> pb.AttachWorkloadMessage - 137, // [137:182] is the sub-list for method output_type - 92, // [92:137] is the sub-list for method input_type - 92, // [92:92] is the sub-list for extension type_name - 92, // [92:92] is the sub-list for extension extendee - 0, // [0:92] is the sub-list for field type_name + 14, // 2: pb.PodResource.nodes_resource:type_name -> pb.NodeResource + 12, // 3: pb.RawParam.string_slice:type_name -> pb.StringSlice + 18, // 4: pb.Networks.networks:type_name -> pb.Network + 90, // 5: pb.Node.cpu:type_name -> pb.Node.CpuEntry + 91, // 6: pb.Node.labels:type_name -> pb.Node.LabelsEntry + 92, // 7: pb.Node.init_cpu:type_name -> pb.Node.InitCpuEntry + 93, // 8: pb.Node.numa:type_name -> pb.Node.NumaEntry + 94, // 9: pb.Node.numa_memory:type_name -> pb.Node.NumaMemoryEntry + 95, // 10: pb.Node.init_volume:type_name -> pb.Node.InitVolumeEntry + 96, // 11: pb.Node.volume:type_name -> pb.Node.VolumeEntry + 97, // 12: pb.Node.init_numa_memory:type_name -> pb.Node.InitNumaMemoryEntry + 20, // 13: pb.Nodes.nodes:type_name -> pb.Node + 98, // 14: pb.SetNodeOptions.delta_cpu:type_name -> pb.SetNodeOptions.DeltaCpuEntry + 99, // 15: pb.SetNodeOptions.delta_numa_memory:type_name -> pb.SetNodeOptions.DeltaNumaMemoryEntry + 100, // 16: pb.SetNodeOptions.numa:type_name -> pb.SetNodeOptions.NumaEntry + 101, // 17: pb.SetNodeOptions.labels:type_name -> pb.SetNodeOptions.LabelsEntry + 102, // 18: pb.SetNodeOptions.delta_volume:type_name -> pb.SetNodeOptions.DeltaVolumeEntry + 0, // 19: pb.SetNodeOptions.bypass_opt:type_name -> pb.TriOpt + 103, // 20: pb.SetNodeOptions.resource_opts:type_name -> pb.SetNodeOptions.ResourceOptsEntry + 104, // 21: pb.NodeFilter.labels:type_name -> pb.NodeFilter.LabelsEntry + 105, // 22: pb.Workload.labels:type_name -> pb.Workload.LabelsEntry + 106, // 23: pb.Workload.publish:type_name -> pb.Workload.PublishEntry + 29, // 24: pb.Workload.status:type_name -> pb.WorkloadStatus + 56, // 25: pb.Workload.resource:type_name -> pb.Resource + 107, // 26: pb.WorkloadStatus.networks:type_name -> pb.WorkloadStatus.NetworksEntry + 29, // 27: pb.WorkloadsStatus.status:type_name -> pb.WorkloadStatus + 29, // 28: pb.SetWorkloadsStatusOptions.status:type_name -> pb.WorkloadStatus + 108, // 29: pb.WorkloadStatusStreamOptions.labels:type_name -> pb.WorkloadStatusStreamOptions.LabelsEntry + 28, // 30: pb.WorkloadStatusStreamMessage.workload:type_name -> pb.Workload + 29, // 31: pb.WorkloadStatusStreamMessage.status:type_name -> pb.WorkloadStatus + 28, // 32: pb.Workloads.workloads:type_name -> pb.Workload + 0, // 33: pb.ReallocOptions.bind_cpu_opt:type_name -> pb.TriOpt + 55, // 34: pb.ReallocOptions.old_resource_opts:type_name -> pb.ResourceOptions + 109, // 35: pb.ReallocOptions.resource_opts:type_name -> pb.ReallocOptions.ResourceOptsEntry + 110, // 36: pb.AddNodeOptions.labels:type_name -> pb.AddNodeOptions.LabelsEntry + 111, // 37: pb.AddNodeOptions.numa:type_name -> pb.AddNodeOptions.NumaEntry + 112, // 38: pb.AddNodeOptions.numa_memory:type_name -> pb.AddNodeOptions.NumaMemoryEntry + 113, // 39: pb.AddNodeOptions.volume_map:type_name -> pb.AddNodeOptions.VolumeMapEntry + 114, // 40: pb.AddNodeOptions.resource_opts:type_name -> pb.AddNodeOptions.ResourceOptsEntry + 115, // 41: pb.GetNodeOptions.labels:type_name -> pb.GetNodeOptions.LabelsEntry + 45, // 42: pb.GetNodeResourceOptions.opts:type_name -> pb.GetNodeOptions + 116, // 43: pb.ListNodesOptions.labels:type_name -> pb.ListNodesOptions.LabelsEntry + 117, // 44: pb.Build.envs:type_name -> pb.Build.EnvsEntry + 118, // 45: pb.Build.args:type_name -> pb.Build.ArgsEntry + 119, // 46: pb.Build.labels:type_name -> pb.Build.LabelsEntry + 120, // 47: pb.Build.artifacts:type_name -> pb.Build.ArtifactsEntry + 121, // 48: pb.Build.cache:type_name -> pb.Build.CacheEntry + 122, // 49: pb.Builds.builds:type_name -> pb.Builds.BuildsEntry + 49, // 50: pb.BuildImageOptions.builds:type_name -> pb.Builds + 2, // 51: pb.BuildImageOptions.build_method:type_name -> pb.BuildImageOptions.BuildMethod + 123, // 52: pb.LogOptions.config:type_name -> pb.LogOptions.ConfigEntry + 53, // 53: pb.EntrypointOptions.log:type_name -> pb.LogOptions + 52, // 54: pb.EntrypointOptions.healthcheck:type_name -> pb.HealthCheckOptions + 51, // 55: pb.EntrypointOptions.hook:type_name -> pb.HookOptions + 124, // 56: pb.EntrypointOptions.sysctls:type_name -> pb.EntrypointOptions.SysctlsEntry + 125, // 57: pb.Resource.cpu:type_name -> pb.Resource.CpuEntry + 126, // 58: pb.Resource.volume_plan_limit:type_name -> pb.Resource.VolumePlanLimitEntry + 127, // 59: pb.Resource.volume_plan_request:type_name -> pb.Resource.VolumePlanRequestEntry + 128, // 60: pb.Volume.volume:type_name -> pb.Volume.VolumeEntry + 54, // 61: pb.DeployOptions.entrypoint:type_name -> pb.EntrypointOptions + 129, // 62: pb.DeployOptions.networks:type_name -> pb.DeployOptions.NetworksEntry + 130, // 63: pb.DeployOptions.labels:type_name -> pb.DeployOptions.LabelsEntry + 131, // 64: pb.DeployOptions.nodelabels:type_name -> pb.DeployOptions.NodelabelsEntry + 3, // 65: pb.DeployOptions.deploy_strategy:type_name -> pb.DeployOptions.Strategy + 132, // 66: pb.DeployOptions.data:type_name -> pb.DeployOptions.DataEntry + 55, // 67: pb.DeployOptions.old_resource_opts:type_name -> pb.ResourceOptions + 27, // 68: pb.DeployOptions.node_filter:type_name -> pb.NodeFilter + 133, // 69: pb.DeployOptions.modes:type_name -> pb.DeployOptions.ModesEntry + 134, // 70: pb.DeployOptions.owners:type_name -> pb.DeployOptions.OwnersEntry + 135, // 71: pb.DeployOptions.resource_opts:type_name -> pb.DeployOptions.ResourceOptsEntry + 58, // 72: pb.ReplaceOptions.deployOpt:type_name -> pb.DeployOptions + 136, // 73: pb.ReplaceOptions.filter_labels:type_name -> pb.ReplaceOptions.FilterLabelsEntry + 137, // 74: pb.ReplaceOptions.copy:type_name -> pb.ReplaceOptions.CopyEntry + 138, // 75: pb.CopyOptions.targets:type_name -> pb.CopyOptions.TargetsEntry + 139, // 76: pb.SendOptions.data:type_name -> pb.SendOptions.DataEntry + 140, // 77: pb.SendOptions.modes:type_name -> pb.SendOptions.ModesEntry + 141, // 78: pb.SendOptions.owners:type_name -> pb.SendOptions.OwnersEntry + 68, // 79: pb.BuildImageMessage.error_detail:type_name -> pb.ErrorDetail + 142, // 80: pb.CreateWorkloadMessage.publish:type_name -> pb.CreateWorkloadMessage.PublishEntry + 56, // 81: pb.CreateWorkloadMessage.resource:type_name -> pb.Resource + 70, // 82: pb.ReplaceWorkloadMessage.create:type_name -> pb.CreateWorkloadMessage + 76, // 83: pb.ReplaceWorkloadMessage.remove:type_name -> pb.RemoveWorkloadMessage + 74, // 84: pb.ListImageMessage.images:type_name -> pb.ImageItem + 1, // 85: pb.AttachWorkloadMessage.std_stream_type:type_name -> pb.StdStreamType + 58, // 86: pb.RunAndWaitOptions.deploy_options:type_name -> pb.DeployOptions + 1, // 87: pb.LogStreamMessage.std_stream_type:type_name -> pb.StdStreamType + 143, // 88: pb.CapacityMessage.node_capacities:type_name -> pb.CapacityMessage.NodeCapacitiesEntry + 13, // 89: pb.SetNodeOptions.ResourceOptsEntry.value:type_name -> pb.RawParam + 13, // 90: pb.ReallocOptions.ResourceOptsEntry.value:type_name -> pb.RawParam + 13, // 91: pb.AddNodeOptions.ResourceOptsEntry.value:type_name -> pb.RawParam + 48, // 92: pb.Builds.BuildsEntry.value:type_name -> pb.Build + 57, // 93: pb.Resource.VolumePlanLimitEntry.value:type_name -> pb.Volume + 57, // 94: pb.Resource.VolumePlanRequestEntry.value:type_name -> pb.Volume + 66, // 95: pb.DeployOptions.ModesEntry.value:type_name -> pb.FileMode + 65, // 96: pb.DeployOptions.OwnersEntry.value:type_name -> pb.FileOwner + 13, // 97: pb.DeployOptions.ResourceOptsEntry.value:type_name -> pb.RawParam + 63, // 98: pb.CopyOptions.TargetsEntry.value:type_name -> pb.CopyPaths + 66, // 99: pb.SendOptions.ModesEntry.value:type_name -> pb.FileMode + 65, // 100: pb.SendOptions.OwnersEntry.value:type_name -> pb.FileOwner + 4, // 101: pb.CoreRPC.Info:input_type -> pb.Empty + 4, // 102: pb.CoreRPC.WatchServiceStatus:input_type -> pb.Empty + 15, // 103: pb.CoreRPC.ListNetworks:input_type -> pb.ListNetworkOptions + 16, // 104: pb.CoreRPC.ConnectNetwork:input_type -> pb.ConnectNetworkOptions + 17, // 105: pb.CoreRPC.DisconnectNetwork:input_type -> pb.DisconnectNetworkOptions + 40, // 106: pb.CoreRPC.AddPod:input_type -> pb.AddPodOptions + 41, // 107: pb.CoreRPC.RemovePod:input_type -> pb.RemovePodOptions + 42, // 108: pb.CoreRPC.GetPod:input_type -> pb.GetPodOptions + 4, // 109: pb.CoreRPC.ListPods:input_type -> pb.Empty + 42, // 110: pb.CoreRPC.GetPodResource:input_type -> pb.GetPodOptions + 42, // 111: pb.CoreRPC.PodResourceStream:input_type -> pb.GetPodOptions + 43, // 112: pb.CoreRPC.AddNode:input_type -> pb.AddNodeOptions + 44, // 113: pb.CoreRPC.RemoveNode:input_type -> pb.RemoveNodeOptions + 47, // 114: pb.CoreRPC.ListPodNodes:input_type -> pb.ListNodesOptions + 47, // 115: pb.CoreRPC.PodNodesStream:input_type -> pb.ListNodesOptions + 45, // 116: pb.CoreRPC.GetNode:input_type -> pb.GetNodeOptions + 45, // 117: pb.CoreRPC.GetNodeEngine:input_type -> pb.GetNodeOptions + 23, // 118: pb.CoreRPC.SetNode:input_type -> pb.SetNodeOptions + 24, // 119: pb.CoreRPC.SetNodeStatus:input_type -> pb.SetNodeStatusOptions + 25, // 120: pb.CoreRPC.GetNodeStatus:input_type -> pb.GetNodeStatusOptions + 4, // 121: pb.CoreRPC.NodeStatusStream:input_type -> pb.Empty + 46, // 122: pb.CoreRPC.GetNodeResource:input_type -> pb.GetNodeResourceOptions + 58, // 123: pb.CoreRPC.CalculateCapacity:input_type -> pb.DeployOptions + 35, // 124: pb.CoreRPC.GetWorkload:input_type -> pb.WorkloadID + 36, // 125: pb.CoreRPC.GetWorkloads:input_type -> pb.WorkloadIDs + 8, // 126: pb.CoreRPC.ListWorkloads:input_type -> pb.ListWorkloadsOptions + 45, // 127: pb.CoreRPC.ListNodeWorkloads:input_type -> pb.GetNodeOptions + 36, // 128: pb.CoreRPC.GetWorkloadsStatus:input_type -> pb.WorkloadIDs + 31, // 129: pb.CoreRPC.SetWorkloadsStatus:input_type -> pb.SetWorkloadsStatusOptions + 32, // 130: pb.CoreRPC.WorkloadStatusStream:input_type -> pb.WorkloadStatusStreamOptions + 64, // 131: pb.CoreRPC.Copy:input_type -> pb.CopyOptions + 67, // 132: pb.CoreRPC.Send:input_type -> pb.SendOptions + 50, // 133: pb.CoreRPC.BuildImage:input_type -> pb.BuildImageOptions + 60, // 134: pb.CoreRPC.CacheImage:input_type -> pb.CacheImageOptions + 61, // 135: pb.CoreRPC.RemoveImage:input_type -> pb.RemoveImageOptions + 62, // 136: pb.CoreRPC.ListImage:input_type -> pb.ListImageOptions + 58, // 137: pb.CoreRPC.CreateWorkload:input_type -> pb.DeployOptions + 59, // 138: pb.CoreRPC.ReplaceWorkload:input_type -> pb.ReplaceOptions + 37, // 139: pb.CoreRPC.RemoveWorkload:input_type -> pb.RemoveWorkloadOptions + 38, // 140: pb.CoreRPC.DissociateWorkload:input_type -> pb.DissociateWorkloadOptions + 83, // 141: pb.CoreRPC.ControlWorkload:input_type -> pb.ControlWorkloadOptions + 87, // 142: pb.CoreRPC.ExecuteWorkload:input_type -> pb.ExecuteWorkloadOptions + 39, // 143: pb.CoreRPC.ReallocResource:input_type -> pb.ReallocOptions + 85, // 144: pb.CoreRPC.LogStream:input_type -> pb.LogStreamOptions + 82, // 145: pb.CoreRPC.RunAndWait:input_type -> pb.RunAndWaitOptions + 5, // 146: pb.CoreRPC.Info:output_type -> pb.CoreInfo + 6, // 147: pb.CoreRPC.WatchServiceStatus:output_type -> pb.ServiceStatus + 19, // 148: pb.CoreRPC.ListNetworks:output_type -> pb.Networks + 18, // 149: pb.CoreRPC.ConnectNetwork:output_type -> pb.Network + 4, // 150: pb.CoreRPC.DisconnectNetwork:output_type -> pb.Empty + 9, // 151: pb.CoreRPC.AddPod:output_type -> pb.Pod + 4, // 152: pb.CoreRPC.RemovePod:output_type -> pb.Empty + 9, // 153: pb.CoreRPC.GetPod:output_type -> pb.Pod + 10, // 154: pb.CoreRPC.ListPods:output_type -> pb.Pods + 11, // 155: pb.CoreRPC.GetPodResource:output_type -> pb.PodResource + 14, // 156: pb.CoreRPC.PodResourceStream:output_type -> pb.NodeResource + 20, // 157: pb.CoreRPC.AddNode:output_type -> pb.Node + 4, // 158: pb.CoreRPC.RemoveNode:output_type -> pb.Empty + 21, // 159: pb.CoreRPC.ListPodNodes:output_type -> pb.Nodes + 20, // 160: pb.CoreRPC.PodNodesStream:output_type -> pb.Node + 20, // 161: pb.CoreRPC.GetNode:output_type -> pb.Node + 7, // 162: pb.CoreRPC.GetNodeEngine:output_type -> pb.Engine + 20, // 163: pb.CoreRPC.SetNode:output_type -> pb.Node + 4, // 164: pb.CoreRPC.SetNodeStatus:output_type -> pb.Empty + 26, // 165: pb.CoreRPC.GetNodeStatus:output_type -> pb.NodeStatusStreamMessage + 26, // 166: pb.CoreRPC.NodeStatusStream:output_type -> pb.NodeStatusStreamMessage + 14, // 167: pb.CoreRPC.GetNodeResource:output_type -> pb.NodeResource + 88, // 168: pb.CoreRPC.CalculateCapacity:output_type -> pb.CapacityMessage + 28, // 169: pb.CoreRPC.GetWorkload:output_type -> pb.Workload + 34, // 170: pb.CoreRPC.GetWorkloads:output_type -> pb.Workloads + 28, // 171: pb.CoreRPC.ListWorkloads:output_type -> pb.Workload + 34, // 172: pb.CoreRPC.ListNodeWorkloads:output_type -> pb.Workloads + 30, // 173: pb.CoreRPC.GetWorkloadsStatus:output_type -> pb.WorkloadsStatus + 30, // 174: pb.CoreRPC.SetWorkloadsStatus:output_type -> pb.WorkloadsStatus + 33, // 175: pb.CoreRPC.WorkloadStatusStream:output_type -> pb.WorkloadStatusStreamMessage + 79, // 176: pb.CoreRPC.Copy:output_type -> pb.CopyMessage + 80, // 177: pb.CoreRPC.Send:output_type -> pb.SendMessage + 69, // 178: pb.CoreRPC.BuildImage:output_type -> pb.BuildImageMessage + 72, // 179: pb.CoreRPC.CacheImage:output_type -> pb.CacheImageMessage + 73, // 180: pb.CoreRPC.RemoveImage:output_type -> pb.RemoveImageMessage + 75, // 181: pb.CoreRPC.ListImage:output_type -> pb.ListImageMessage + 70, // 182: pb.CoreRPC.CreateWorkload:output_type -> pb.CreateWorkloadMessage + 71, // 183: pb.CoreRPC.ReplaceWorkload:output_type -> pb.ReplaceWorkloadMessage + 76, // 184: pb.CoreRPC.RemoveWorkload:output_type -> pb.RemoveWorkloadMessage + 77, // 185: pb.CoreRPC.DissociateWorkload:output_type -> pb.DissociateWorkloadMessage + 84, // 186: pb.CoreRPC.ControlWorkload:output_type -> pb.ControlWorkloadMessage + 81, // 187: pb.CoreRPC.ExecuteWorkload:output_type -> pb.AttachWorkloadMessage + 78, // 188: pb.CoreRPC.ReallocResource:output_type -> pb.ReallocResourceMessage + 86, // 189: pb.CoreRPC.LogStream:output_type -> pb.LogStreamMessage + 81, // 190: pb.CoreRPC.RunAndWait:output_type -> pb.AttachWorkloadMessage + 146, // [146:191] is the sub-list for method output_type + 101, // [101:146] is the sub-list for method input_type + 101, // [101:101] is the sub-list for extension type_name + 101, // [101:101] is the sub-list for extension extendee + 0, // [0:101] is the sub-list for field type_name } func init() { file_rpc_gen_core_proto_init() } @@ -7887,7 +8192,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NodeResource); i { + switch v := v.(*StringSlice); i { case 0: return &v.state case 1: @@ -7899,7 +8204,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListNetworkOptions); i { + switch v := v.(*RawParam); i { case 0: return &v.state case 1: @@ -7911,7 +8216,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConnectNetworkOptions); i { + switch v := v.(*NodeResource); i { case 0: return &v.state case 1: @@ -7923,7 +8228,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DisconnectNetworkOptions); i { + switch v := v.(*ListNetworkOptions); i { case 0: return &v.state case 1: @@ -7935,7 +8240,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Network); i { + switch v := v.(*ConnectNetworkOptions); i { case 0: return &v.state case 1: @@ -7947,7 +8252,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Networks); i { + switch v := v.(*DisconnectNetworkOptions); i { case 0: return &v.state case 1: @@ -7959,7 +8264,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Node); i { + switch v := v.(*Network); i { case 0: return &v.state case 1: @@ -7971,7 +8276,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Nodes); i { + switch v := v.(*Networks); i { case 0: return &v.state case 1: @@ -7983,7 +8288,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NodeAvailable); i { + switch v := v.(*Node); i { case 0: return &v.state case 1: @@ -7995,7 +8300,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetNodeOptions); i { + switch v := v.(*Nodes); i { case 0: return &v.state case 1: @@ -8007,7 +8312,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetNodeStatusOptions); i { + switch v := v.(*NodeAvailable); i { case 0: return &v.state case 1: @@ -8019,7 +8324,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetNodeStatusOptions); i { + switch v := v.(*SetNodeOptions); i { case 0: return &v.state case 1: @@ -8031,7 +8336,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NodeStatusStreamMessage); i { + switch v := v.(*SetNodeStatusOptions); i { case 0: return &v.state case 1: @@ -8043,7 +8348,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NodeFilter); i { + switch v := v.(*GetNodeStatusOptions); i { case 0: return &v.state case 1: @@ -8055,7 +8360,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Workload); i { + switch v := v.(*NodeStatusStreamMessage); i { case 0: return &v.state case 1: @@ -8067,7 +8372,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkloadStatus); i { + switch v := v.(*NodeFilter); i { case 0: return &v.state case 1: @@ -8079,7 +8384,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkloadsStatus); i { + switch v := v.(*Workload); i { case 0: return &v.state case 1: @@ -8091,7 +8396,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetWorkloadsStatusOptions); i { + switch v := v.(*WorkloadStatus); i { case 0: return &v.state case 1: @@ -8103,7 +8408,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkloadStatusStreamOptions); i { + switch v := v.(*WorkloadsStatus); i { case 0: return &v.state case 1: @@ -8115,7 +8420,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkloadStatusStreamMessage); i { + switch v := v.(*SetWorkloadsStatusOptions); i { case 0: return &v.state case 1: @@ -8127,7 +8432,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Workloads); i { + switch v := v.(*WorkloadStatusStreamOptions); i { case 0: return &v.state case 1: @@ -8139,7 +8444,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkloadID); i { + switch v := v.(*WorkloadStatusStreamMessage); i { case 0: return &v.state case 1: @@ -8151,7 +8456,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkloadIDs); i { + switch v := v.(*Workloads); i { case 0: return &v.state case 1: @@ -8163,7 +8468,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RemoveWorkloadOptions); i { + switch v := v.(*WorkloadID); i { case 0: return &v.state case 1: @@ -8175,7 +8480,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DissociateWorkloadOptions); i { + switch v := v.(*WorkloadIDs); i { case 0: return &v.state case 1: @@ -8187,7 +8492,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReallocOptions); i { + switch v := v.(*RemoveWorkloadOptions); i { case 0: return &v.state case 1: @@ -8199,7 +8504,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AddPodOptions); i { + switch v := v.(*DissociateWorkloadOptions); i { case 0: return &v.state case 1: @@ -8211,7 +8516,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RemovePodOptions); i { + switch v := v.(*ReallocOptions); i { case 0: return &v.state case 1: @@ -8223,7 +8528,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetPodOptions); i { + switch v := v.(*AddPodOptions); i { case 0: return &v.state case 1: @@ -8235,7 +8540,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AddNodeOptions); i { + switch v := v.(*RemovePodOptions); i { case 0: return &v.state case 1: @@ -8247,7 +8552,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RemoveNodeOptions); i { + switch v := v.(*GetPodOptions); i { case 0: return &v.state case 1: @@ -8259,7 +8564,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetNodeOptions); i { + switch v := v.(*AddNodeOptions); i { case 0: return &v.state case 1: @@ -8271,7 +8576,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetNodeResourceOptions); i { + switch v := v.(*RemoveNodeOptions); i { case 0: return &v.state case 1: @@ -8283,7 +8588,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListNodesOptions); i { + switch v := v.(*GetNodeOptions); i { case 0: return &v.state case 1: @@ -8295,7 +8600,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Build); i { + switch v := v.(*GetNodeResourceOptions); i { case 0: return &v.state case 1: @@ -8307,7 +8612,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Builds); i { + switch v := v.(*ListNodesOptions); i { case 0: return &v.state case 1: @@ -8319,7 +8624,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BuildImageOptions); i { + switch v := v.(*Build); i { case 0: return &v.state case 1: @@ -8331,7 +8636,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*HookOptions); i { + switch v := v.(*Builds); i { case 0: return &v.state case 1: @@ -8343,7 +8648,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*HealthCheckOptions); i { + switch v := v.(*BuildImageOptions); i { case 0: return &v.state case 1: @@ -8355,7 +8660,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogOptions); i { + switch v := v.(*HookOptions); i { case 0: return &v.state case 1: @@ -8367,7 +8672,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EntrypointOptions); i { + switch v := v.(*HealthCheckOptions); i { case 0: return &v.state case 1: @@ -8379,7 +8684,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResourceOptions); i { + switch v := v.(*LogOptions); i { case 0: return &v.state case 1: @@ -8391,7 +8696,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Resource); i { + switch v := v.(*EntrypointOptions); i { case 0: return &v.state case 1: @@ -8403,7 +8708,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Volume); i { + switch v := v.(*ResourceOptions); i { case 0: return &v.state case 1: @@ -8415,7 +8720,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeployOptions); i { + switch v := v.(*Resource); i { case 0: return &v.state case 1: @@ -8427,7 +8732,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReplaceOptions); i { + switch v := v.(*Volume); i { case 0: return &v.state case 1: @@ -8439,7 +8744,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CacheImageOptions); i { + switch v := v.(*DeployOptions); i { case 0: return &v.state case 1: @@ -8451,7 +8756,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RemoveImageOptions); i { + switch v := v.(*ReplaceOptions); i { case 0: return &v.state case 1: @@ -8463,7 +8768,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListImageOptions); i { + switch v := v.(*CacheImageOptions); i { case 0: return &v.state case 1: @@ -8475,7 +8780,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CopyPaths); i { + switch v := v.(*RemoveImageOptions); i { case 0: return &v.state case 1: @@ -8487,7 +8792,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CopyOptions); i { + switch v := v.(*ListImageOptions); i { case 0: return &v.state case 1: @@ -8499,7 +8804,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FileOwner); i { + switch v := v.(*CopyPaths); i { case 0: return &v.state case 1: @@ -8511,7 +8816,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FileMode); i { + switch v := v.(*CopyOptions); i { case 0: return &v.state case 1: @@ -8523,7 +8828,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SendOptions); i { + switch v := v.(*FileOwner); i { case 0: return &v.state case 1: @@ -8535,7 +8840,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ErrorDetail); i { + switch v := v.(*FileMode); i { case 0: return &v.state case 1: @@ -8547,7 +8852,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BuildImageMessage); i { + switch v := v.(*SendOptions); i { case 0: return &v.state case 1: @@ -8559,7 +8864,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateWorkloadMessage); i { + switch v := v.(*ErrorDetail); i { case 0: return &v.state case 1: @@ -8571,7 +8876,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReplaceWorkloadMessage); i { + switch v := v.(*BuildImageMessage); i { case 0: return &v.state case 1: @@ -8583,7 +8888,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CacheImageMessage); i { + switch v := v.(*CreateWorkloadMessage); i { case 0: return &v.state case 1: @@ -8595,7 +8900,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RemoveImageMessage); i { + switch v := v.(*ReplaceWorkloadMessage); i { case 0: return &v.state case 1: @@ -8607,7 +8912,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ImageItem); i { + switch v := v.(*CacheImageMessage); i { case 0: return &v.state case 1: @@ -8619,7 +8924,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListImageMessage); i { + switch v := v.(*RemoveImageMessage); i { case 0: return &v.state case 1: @@ -8631,7 +8936,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RemoveWorkloadMessage); i { + switch v := v.(*ImageItem); i { case 0: return &v.state case 1: @@ -8643,7 +8948,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DissociateWorkloadMessage); i { + switch v := v.(*ListImageMessage); i { case 0: return &v.state case 1: @@ -8655,7 +8960,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReallocResourceMessage); i { + switch v := v.(*RemoveWorkloadMessage); i { case 0: return &v.state case 1: @@ -8667,7 +8972,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CopyMessage); i { + switch v := v.(*DissociateWorkloadMessage); i { case 0: return &v.state case 1: @@ -8679,7 +8984,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SendMessage); i { + switch v := v.(*ReallocResourceMessage); i { case 0: return &v.state case 1: @@ -8691,7 +8996,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AttachWorkloadMessage); i { + switch v := v.(*CopyMessage); i { case 0: return &v.state case 1: @@ -8703,7 +9008,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RunAndWaitOptions); i { + switch v := v.(*SendMessage); i { case 0: return &v.state case 1: @@ -8715,7 +9020,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ControlWorkloadOptions); i { + switch v := v.(*AttachWorkloadMessage); i { case 0: return &v.state case 1: @@ -8727,7 +9032,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ControlWorkloadMessage); i { + switch v := v.(*RunAndWaitOptions); i { case 0: return &v.state case 1: @@ -8739,7 +9044,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogStreamOptions); i { + switch v := v.(*ControlWorkloadOptions); i { case 0: return &v.state case 1: @@ -8751,7 +9056,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogStreamMessage); i { + switch v := v.(*ControlWorkloadMessage); i { case 0: return &v.state case 1: @@ -8763,7 +9068,7 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecuteWorkloadOptions); i { + switch v := v.(*LogStreamOptions); i { case 0: return &v.state case 1: @@ -8775,6 +9080,30 @@ func file_rpc_gen_core_proto_init() { } } file_rpc_gen_core_proto_msgTypes[82].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_rpc_gen_core_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ExecuteWorkloadOptions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_rpc_gen_core_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CapacityMessage); i { case 0: return &v.state @@ -8787,13 +9116,17 @@ func file_rpc_gen_core_proto_init() { } } } + file_rpc_gen_core_proto_msgTypes[9].OneofWrappers = []interface{}{ + (*RawParam_Str)(nil), + (*RawParam_StringSlice)(nil), + } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_rpc_gen_core_proto_rawDesc, NumEnums: 4, - NumMessages: 134, + NumMessages: 140, NumExtensions: 0, NumServices: 1, }, diff --git a/rpc/gen/core.proto b/rpc/gen/core.proto index 1f3a72865..c435e104e 100644 --- a/rpc/gen/core.proto +++ b/rpc/gen/core.proto @@ -105,13 +105,27 @@ message PodResource { repeated NodeResource nodes_resource = 2; } +message StringSlice { + repeated string slice = 1; +} + +message RawParam { + oneof value { + string str = 1; + StringSlice string_slice = 2; + } +} + + message NodeResource { string name = 1; - double cpu_percent = 2; - double memory_percent = 3; - double storage_percent = 4; - double volume_percent = 5; + double cpu_percent = 2; + double memory_percent = 3; + double storage_percent = 4; + double volume_percent = 5; repeated string diffs = 6; + string resource_capacity = 7; + string resource_usage = 8; } message ListNetworkOptions { @@ -164,6 +178,8 @@ message Node { int64 volume_used = 20; map init_numa_memory = 21; bool bypass = 22; + string resource_capacity = 23; + string resource_usage = 24; } message Nodes { @@ -190,6 +206,8 @@ message SetNodeOptions { string ca = 13; string cert = 14; string key = 15; + map resource_opts = 16; + bool delta = 17; } message SetNodeStatusOptions { @@ -228,6 +246,7 @@ message Workload { Resource resource = 10; int64 create_time = 11; repeated string env = 12; + string resource_args = 13; } message WorkloadStatus { @@ -237,7 +256,7 @@ message WorkloadStatus { map networks = 4; bytes extension = 5; int64 ttl = 6; - // extra fields used to set workload status + // extra fields used to set workload status string appname = 7; string nodename = 8; string entrypoint = 9; @@ -295,8 +314,9 @@ enum TriOpt { message ReallocOptions { string id = 1; - TriOpt bind_cpu_opt = 2; - ResourceOptions resource_opts = 3; + TriOpt bind_cpu_opt = 2; // deprecated + ResourceOptions old_resource_opts = 3; + map resource_opts = 4; } message AddPodOptions { @@ -327,6 +347,7 @@ message AddNodeOptions { map numa_memory = 12; int64 storage = 13; map volume_map = 14; + map resource_opts = 15; } message RemoveNodeOptions { @@ -424,6 +445,7 @@ message EntrypointOptions { repeated string commands = 11; } +// deprecated message ResourceOptions { double cpu_quota_limit = 1; double cpu_quota_request = 2; @@ -436,6 +458,7 @@ message ResourceOptions { repeated string volumes_request = 9; } +// deprecated message Resource { double cpu_quota_limit = 1; double cpu_quota_request = 2; @@ -484,11 +507,12 @@ message DeployOptions { bool ignore_hook = 21; repeated string after_create = 22; bytes raw_args = 23; - ResourceOptions resource_opts = 24; + ResourceOptions old_resource_opts = 24; NodeFilter node_filter = 25; // should be part of field no.19 map modes = 26; map owners = 27; + map resource_opts = 28; } message ReplaceOptions { @@ -568,6 +592,7 @@ message CreateWorkloadMessage { map publish = 7; bytes hook = 8; Resource resource = 9; + string resource_args = 10; } message ReplaceWorkloadMessage { diff --git a/rpc/gen/core_grpc.pb.go b/rpc/gen/core_grpc.pb.go index a8aaf52b6..239d8ac32 100644 --- a/rpc/gen/core_grpc.pb.go +++ b/rpc/gen/core_grpc.pb.go @@ -1,8 +1,4 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. -// versions: -// - protoc-gen-go-grpc v1.2.0 -// - protoc v3.6.1 -// source: rpc/gen/core.proto package pb diff --git a/rpc/rpc.go b/rpc/rpc.go index d2e830e76..dd52f72bd 100644 --- a/rpc/rpc.go +++ b/rpc/rpc.go @@ -884,31 +884,13 @@ func (v *Vibranium) ReallocResource(ctx context.Context, opts *pb.ReallocOptions return msg, grpcstatus.Errorf(ReallocResource, "%v", types.ErrNoWorkloadIDs) } - vbsRequest, err := types.NewVolumeBindings(opts.ResourceOpts.VolumesRequest) - if err != nil { - return msg, grpcstatus.Error(ReallocResource, err.Error()) - } - - vbsLimit, err := types.NewVolumeBindings(opts.ResourceOpts.VolumesLimit) - if err != nil { - return msg, grpcstatus.Error(ReallocResource, err.Error()) - } + fillRPCNewReallocOptions(opts) if err := v.cluster.ReallocResource( - task.context, + 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 + vbsRequest.TotalSize(), - StorageLimit: opts.ResourceOpts.StorageLimit + vbsLimit.TotalSize(), - }, + ID: opts.Id, + ResourceOpts: toCoreRawParams(opts.ResourceOpts), }, ); err != nil { return msg, grpcstatus.Error(ReallocResource, err.Error()) diff --git a/rpc/rpc_test.go b/rpc/rpc_test.go index 2e92f70c6..cb78575a3 100644 --- a/rpc/rpc_test.go +++ b/rpc/rpc_test.go @@ -65,11 +65,9 @@ func TestAddNode(t *testing.T) { func TestSetNodeTranform(t *testing.T) { b := &pb.SetNodeOptions{ Nodename: "a", - DeltaCpu: map[string]int32{"0": 1, "1": -1}, } - o, err := toCoreSetNodeOptions(b) + _, err := toCoreSetNodeOptions(b) assert.Nil(t, err) - assert.Equal(t, 2, len(o.DeltaCPU)) } func TestRunAndWaitSync(t *testing.T) { @@ -87,7 +85,7 @@ func TestRunAndWaitSync(t *testing.T) { Podname: "pod", Image: "image", OpenStdin: false, - ResourceOpts: &pb.ResourceOptions{}, + ResourceOpts: map[string]*pb.RawParam{}, }, Cmd: []byte("ping"), Async: false, @@ -146,7 +144,7 @@ func TestRunAndWaitAsync(t *testing.T) { Podname: "pod", Image: "image", OpenStdin: false, - ResourceOpts: &pb.ResourceOptions{}, + ResourceOpts: map[string]*pb.RawParam{}, }, Cmd: []byte("ping"), Async: true, diff --git a/rpc/transform.go b/rpc/transform.go index af5775d2b..d3b5ee77c 100644 --- a/rpc/transform.go +++ b/rpc/transform.go @@ -2,9 +2,12 @@ package rpc import ( "bytes" + "encoding/json" "fmt" "time" + "golang.org/x/exp/maps" + enginetypes "github.com/projecteru2/core/engine/types" "github.com/projecteru2/core/log" pb "github.com/projecteru2/core/rpc/gen" @@ -21,14 +24,6 @@ func toRPCServiceStatus(status types.ServiceStatus) *pb.ServiceStatus { } } -func toRPCCPUMap(m types.CPUMap) map[string]int32 { - cpu := make(map[string]int32) - for label, value := range m { - cpu[label] = int32(value) - } - return cpu -} - func toRPCPod(p *types.Pod) *pb.Pod { return &pb.Pod{Name: p.Name, Desc: p.Desc} } @@ -38,30 +33,34 @@ func toRPCNetwork(n *enginetypes.Network) *pb.Network { } func toRPCNode(n *types.Node) *pb.Node { - return &pb.Node{ - Name: n.Name, - Endpoint: n.Endpoint, - Podname: n.Podname, - Cpu: toRPCCPUMap(n.CPU), - CpuUsed: n.CPUUsed, - Memory: n.MemCap, - MemoryUsed: n.InitMemCap - n.MemCap, - Storage: n.StorageCap, - StorageUsed: n.StorageUsed(), - Volume: n.Volume, - VolumeUsed: n.VolumeUsed, - Available: n.Available, - Labels: n.Labels, - InitCpu: toRPCCPUMap(n.InitCPU), - InitMemory: n.InitMemCap, - InitStorage: n.InitStorageCap, - InitVolume: n.InitVolume, - Info: n.NodeInfo, - Numa: n.NUMA, - NumaMemory: n.NUMAMemory, - InitNumaMemory: n.InitNUMAMemory, - Bypass: n.Bypass, + resourceCapacity := map[string]types.RawParams{} + resourceUsage := map[string]types.RawParams{} + + for plugin, args := range n.ResourceCapacity { + resourceCapacity[plugin] = types.RawParams(args) + } + for plugin, args := range n.ResourceUsage { + resourceUsage[plugin] = types.RawParams(args) } + + node := &pb.Node{ + Name: n.Name, + Endpoint: n.Endpoint, + Podname: n.Podname, + Available: n.Available, + Labels: n.Labels, + Info: n.NodeInfo, + Bypass: n.Bypass, + ResourceCapacity: toRPCResourceArgs(resourceCapacity), + ResourceUsage: toRPCResourceArgs(resourceUsage), + } + fillOldNodeMeta(node, resourceCapacity, resourceUsage) + return node +} + +func toRPCResourceArgs(v interface{}) string { + body, _ := json.Marshal(v) + return string(body) } func toRPCEngine(e *enginetypes.Info) *pb.Engine { @@ -71,13 +70,28 @@ func toRPCEngine(e *enginetypes.Info) *pb.Engine { } func toRPCNodeResource(nr *types.NodeResource) *pb.NodeResource { + resourceCapacity := map[string]types.RawParams{} + resourceUsage := map[string]types.RawParams{} + + for plugin, args := range nr.ResourceCapacity { + resourceCapacity[plugin] = types.RawParams(args) + } + for plugin, args := range nr.ResourceUsage { + resourceUsage[plugin] = types.RawParams(args) + } + + node := &pb.Node{} + fillOldNodeMeta(node, resourceCapacity, resourceUsage) + return &pb.NodeResource{ - Name: nr.Name, - CpuPercent: nr.CPUPercent, - MemoryPercent: nr.MemoryPercent, - StoragePercent: nr.StoragePercent, - VolumePercent: nr.VolumePercent, - Diffs: nr.Diffs, + Name: nr.Name, + Diffs: nr.Diffs, + CpuPercent: node.CpuUsed / float64(len(node.InitCpu)), + MemoryPercent: float64(node.MemoryUsed) / float64(node.InitMemory), + StoragePercent: float64(node.StorageUsed) / float64(node.InitStorage), + VolumePercent: float64(node.VolumeUsed) / float64(utils.Sum(maps.Values(node.InitVolume))), + ResourceCapacity: toRPCResourceArgs(resourceCapacity), + ResourceUsage: toRPCResourceArgs(resourceUsage), } } @@ -131,44 +145,66 @@ func toCoreSendOptions(b *pb.SendOptions) (*types.SendOptions, error) { // nolin } func toCoreAddNodeOptions(b *pb.AddNodeOptions) *types.AddNodeOptions { + if b.ResourceOpts == nil { + b.ResourceOpts = map[string]*pb.RawParam{ + "cpu": newPBRawParamStr(fmt.Sprintf("%v", b.Cpu)), + "share": newPBRawParamStr(fmt.Sprintf("%v", b.Share)), + "memory": newPBRawParamStr(fmt.Sprintf("%v", b.Memory)), + "numa-cpu": newPBRawParamStringSlice(maps.Values(b.Numa)), + "numa-memory": newPBRawParamStringSlice(utils.Map(maps.Values(b.NumaMemory), func(v int64) string { + return fmt.Sprintf("%v", v) + })), + "storage": newPBRawParamStr(fmt.Sprintf("%v", b.Storage)), + } + volumes := []string{} + for device, size := range b.VolumeMap { + volumes = append(volumes, fmt.Sprintf("%v:%v", device, size)) + } + b.ResourceOpts["volumes"] = newPBRawParamStringSlice(volumes) + } + r := &types.AddNodeOptions{ - Nodename: b.Nodename, - Endpoint: b.Endpoint, - Podname: b.Podname, - Ca: b.Ca, - Cert: b.Cert, - Key: b.Key, - CPU: int(b.Cpu), - Share: int(b.Share), - Memory: b.Memory, - Storage: b.Storage, - Labels: b.Labels, - Numa: b.Numa, - NumaMemory: b.NumaMemory, - Volume: b.VolumeMap, + Nodename: b.Nodename, + Endpoint: b.Endpoint, + Podname: b.Podname, + Ca: b.Ca, + Cert: b.Cert, + Key: b.Key, + Labels: b.Labels, + ResourceOpts: toCoreRawParams(b.ResourceOpts), } return r } func toCoreSetNodeOptions(b *pb.SetNodeOptions) (*types.SetNodeOptions, error) { // nolint + if b.ResourceOpts == nil { + b.Delta = true + b.ResourceOpts = map[string]*pb.RawParam{ + "cpu": newPBRawParamStr(fmt.Sprintf("%v", b.DeltaCpu)), + "memory": newPBRawParamStr(fmt.Sprintf("%v", b.DeltaMemory)), + "numa-cpu": newPBRawParamStringSlice(maps.Values(b.Numa)), + "numa-memory": newPBRawParamStringSlice(utils.Map(maps.Values(b.DeltaNumaMemory), func(v int64) string { + return fmt.Sprintf("%v", v) + })), + "storage": newPBRawParamStr(fmt.Sprintf("%v", b.DeltaStorage)), + } + volumes := []string{} + for device, size := range b.DeltaVolume { + volumes = append(volumes, fmt.Sprintf("%v:%v", device, size)) + } + b.ResourceOpts["volumes"] = newPBRawParamStringSlice(volumes) + } r := &types.SetNodeOptions{ - Nodename: b.Nodename, - Endpoint: b.Endpoint, - WorkloadsDown: b.WorkloadsDown, - DeltaCPU: types.CPUMap{}, - DeltaMemory: b.DeltaMemory, - DeltaStorage: b.DeltaStorage, - DeltaNUMAMemory: b.DeltaNumaMemory, - DeltaVolume: b.DeltaVolume, - NUMA: b.Numa, - Labels: b.Labels, - BypassOpt: types.TriOptions(b.BypassOpt), - Ca: b.Ca, - Cert: b.Cert, - Key: b.Key, - } - for cpuID, cpuShare := range b.DeltaCpu { - r.DeltaCPU[cpuID] = int64(cpuShare) + Nodename: b.Nodename, + Endpoint: b.Endpoint, + Ca: b.Ca, + Cert: b.Cert, + Key: b.Key, + WorkloadsDown: b.WorkloadsDown, + ResourceOpts: toCoreRawParams(b.ResourceOpts), + Delta: b.Delta, + Labels: b.Labels, + BypassOpt: types.TriOptions(b.BypassOpt), } return r, nil } @@ -245,6 +281,13 @@ func toCoreReplaceOptions(r *pb.ReplaceOptions) (*types.ReplaceOptions, error) { } func toCoreDeployOptions(d *pb.DeployOptions) (*types.DeployOptions, error) { + if d.ResourceOpts == nil { + d.ResourceOpts = toNewResourceOptions(d.OldResourceOpts) + if d.OldResourceOpts.CpuBind { + d.ResourceOpts["cpu-bind"] = newPBRawParamStr("true") + } + } + if d.Entrypoint == nil || d.Entrypoint.Name == "" { return nil, types.ErrNoEntryInSpec } @@ -285,7 +328,6 @@ func toCoreDeployOptions(d *pb.DeployOptions) (*types.DeployOptions, error) { entry.Hook.Force = entrypoint.Hook.Force } - var err error files := []types.LinuxFile{} for filename, bs := range d.Data { file := types.LinuxFile{ @@ -301,16 +343,6 @@ func toCoreDeployOptions(d *pb.DeployOptions) (*types.DeployOptions, error) { files = append(files, file) } - vbsLimit, err := types.NewVolumeBindings(d.ResourceOpts.VolumesLimit) - if err != nil { - return nil, err - } - - vbsRequest, err := types.NewVolumeBindings(d.ResourceOpts.VolumesRequest) - if err != nil { - return nil, err - } - nf := types.NodeFilter{ Podname: d.Podname, Includes: d.Nodenames, @@ -323,17 +355,7 @@ func toCoreDeployOptions(d *pb.DeployOptions) (*types.DeployOptions, error) { } 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, - }, + ResourceOpts: toCoreRawParams(d.ResourceOpts), Name: d.Name, Entrypoint: entry, Podname: d.Podname, @@ -362,25 +384,22 @@ func toRPCCreateWorkloadMessage(c *types.CreateWorkloadMessage) *pb.CreateWorklo if c == nil { return nil } + + resourceArgs := map[string]types.RawParams{} + for plugin, args := range c.ResourceArgs { + resourceArgs[plugin] = types.RawParams(args) + } + msg := &pb.CreateWorkloadMessage{ - Podname: c.Podname, - Nodename: c.Nodename, - Id: c.WorkloadID, - Name: c.WorkloadName, - 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), - }, + Podname: c.Podname, + Nodename: c.Nodename, + Id: c.WorkloadID, + Name: c.WorkloadName, + Success: c.Error == nil, + Publish: utils.EncodePublishInfo(c.Publish), + Hook: utils.MergeHookOutputs(c.Hook), + ResourceArgs: toRPCResourceArgs(resourceArgs), + Resource: toRPCWorkloadResource(resourceArgs), } if c.Error != nil { msg.Error = c.Error.Error() @@ -520,44 +539,26 @@ func toRPCWorkload(ctx context.Context, c *types.Workload) (*pb.Workload, error) utils.MakePublishInfo(c.StatusMeta.Networks, meta.Publish), ) } - return &pb.Workload{ - Id: c.ID, - Podname: c.Podname, - Nodename: c.Nodename, - Name: c.Name, - Privileged: c.Privileged, - Publish: publish, - Image: c.Image, - Labels: c.Labels, - Status: toRPCWorkloadStatus(c.StatusMeta), - CreateTime: c.CreateTime, - 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), - }, - Env: c.Env, - }, nil -} - -func toRPCVolumePlan(v types.VolumePlan) map[string]*pb.Volume { - if v == nil { - return nil + resourceArgs := map[string]types.RawParams{} + for plugin, args := range c.ResourceArgs { + resourceArgs[plugin] = types.RawParams(args) } - msg := map[string]*pb.Volume{} - for vb, volume := range v { - msg[vb.ToString(false)] = &pb.Volume{Volume: volume} - } - return msg + return &pb.Workload{ + Id: c.ID, + Podname: c.Podname, + Nodename: c.Nodename, + Name: c.Name, + Privileged: c.Privileged, + Publish: publish, + Image: c.Image, + Labels: c.Labels, + Status: toRPCWorkloadStatus(c.StatusMeta), + CreateTime: c.CreateTime, + ResourceArgs: toRPCResourceArgs(resourceArgs), + Resource: toRPCWorkloadResource(resourceArgs), + Env: c.Env, + }, nil } func toRPCLogStreamMessage(msg *types.LogStreamMessage) *pb.LogStreamMessage { @@ -616,6 +617,26 @@ func toCoreRemoveImageOptions(opts *pb.RemoveImageOptions) *types.ImageOptions { } } +func toCoreRawParams(params map[string]*pb.RawParam) map[string]interface{} { + if params == nil { + return nil + } + res := map[string]interface{}{} + for key, param := range params { + if param.Value == nil { + res[key] = nil + continue + } + switch param.Value.(type) { + case *pb.RawParam_Str: + res[key] = param.GetStr() + case *pb.RawParam_StringSlice: + res[key] = param.GetStringSlice().Slice + } + } + return res +} + func toRPCListImageMessage(msg *types.ListImageMessage) *pb.ListImageMessage { m := &pb.ListImageMessage{ Images: []*pb.ImageItem{}, @@ -648,3 +669,119 @@ func toCoreListImageOptions(opts *pb.ListImageOptions) *types.ImageOptions { Filter: opts.Filter, } } + +func newPBRawParamStr(str string) *pb.RawParam { + return &pb.RawParam{ + Value: &pb.RawParam_Str{ + Str: str, + }, + } +} + +func newPBRawParamStringSlice(strs []string) *pb.RawParam { + return &pb.RawParam{ + Value: &pb.RawParam_StringSlice{ + StringSlice: &pb.StringSlice{ + Slice: strs, + }, + }, + } +} + +// fillOldNodeMeta fills the old node meta based on the new node resource args. +// uses some hard code, should be removed in the future. +func fillOldNodeMeta(node *pb.Node, resourceCapacity map[string]types.RawParams, resourceUsage map[string]types.RawParams) { + if capacity, ok := resourceCapacity["cpumem"]; ok { + usage := resourceUsage["cpumem"] + node.Cpu = map[string]int32{} + node.InitCpu = types.ConvertRawParamsToMap[int32](capacity.RawParams("cpu_map")) + node.CpuUsed = usage.Float64("cpu") + node.InitMemory = capacity.Int64("memory") + node.MemoryUsed = usage.Int64("memory") + node.Memory = node.InitMemory - node.MemoryUsed + node.Numa = types.ConvertRawParamsToMap[string](usage.RawParams("numa")) + node.InitNumaMemory = types.ConvertRawParamsToMap[int64](capacity.RawParams("numa_memory")) + node.NumaMemory = map[string]int64{} + + cpuMapUsed := types.ConvertRawParamsToMap[int32](usage.RawParams("cpu_map")) + for cpuID := range node.InitCpu { + node.Cpu[cpuID] = node.InitCpu[cpuID] - cpuMapUsed[cpuID] + } + + numaMemoryUsed := types.ConvertRawParamsToMap[int64](usage.RawParams("numa_memory")) + for numaNodeID := range numaMemoryUsed { + node.NumaMemory[numaNodeID] = node.InitNumaMemory[numaNodeID] - numaMemoryUsed[numaNodeID] + } + } + + if capacity, ok := resourceCapacity["volume"]; ok { + usage := resourceUsage["volume"] + node.InitStorage = capacity.Int64("storage") + node.StorageUsed = usage.Int64("storage") + node.Storage = node.InitStorage - node.StorageUsed + node.InitVolume = types.ConvertRawParamsToMap[int64](capacity.RawParams("volumes")) + node.Volume = map[string]int64{} + + volumeUsed := types.ConvertRawParamsToMap[int64](usage.RawParams("volumes")) + for device, size := range volumeUsed { + node.Volume[device] = node.InitVolume[device] - size + node.VolumeUsed += size + } + } +} + +func toRPCWorkloadResource(resourceArgs map[string]types.RawParams) *pb.Resource { + res := &pb.Resource{} + if args, ok := resourceArgs["cpumem"]; ok { + res.CpuQuotaRequest = args.Float64("cpu_request") + res.CpuQuotaLimit = args.Float64("cpu_limit") + res.Cpu = types.ConvertRawParamsToMap[int32](args.RawParams("cpu_map")) + res.MemoryRequest = args.Int64("memory_request") + res.MemoryLimit = args.Int64("memory_limit") + } + + if args, ok := resourceArgs["volume"]; ok { + res.VolumesRequest = args.StringSlice("volumes_request") + res.VolumesLimit = args.StringSlice("volumes_limit") + res.VolumePlanRequest = toRPCVolumePlan(types.ConvertRawParamsToMap[map[string]int64](args.RawParams("volume_plan_request"))) + res.VolumePlanLimit = toRPCVolumePlan(types.ConvertRawParamsToMap[map[string]int64](args.RawParams("volume_plan_limit"))) + res.StorageRequest = args.Int64("storage_request") + res.StorageLimit = args.Int64("storage_limit") + } + return res +} + +func toRPCVolumePlan(volumePlan map[string]map[string]int64) map[string]*pb.Volume { + res := map[string]*pb.Volume{} + for binding, plan := range volumePlan { + res[binding] = &pb.Volume{ + Volume: plan, + } + } + return res +} + +func toNewResourceOptions(opts *pb.ResourceOptions) map[string]*pb.RawParam { + return map[string]*pb.RawParam{ + "cpu-request": newPBRawParamStr(fmt.Sprintf("%v", opts.CpuQuotaRequest)), + "cpu-limit": newPBRawParamStr(fmt.Sprintf("%v", opts.CpuQuotaLimit)), + "memory-request": newPBRawParamStr(fmt.Sprintf("%v", opts.MemoryRequest)), + "memory-limit": newPBRawParamStr(fmt.Sprintf("%v", opts.MemoryLimit)), + "storage-request": newPBRawParamStr(fmt.Sprintf("%v", opts.StorageRequest)), + "storage-limit": newPBRawParamStr(fmt.Sprintf("%v", opts.StorageLimit)), + "volumes-request": newPBRawParamStringSlice(opts.VolumesRequest), + "volumes-limit": newPBRawParamStringSlice(opts.VolumesLimit), + } +} + +func fillRPCNewReallocOptions(opts *pb.ReallocOptions) { + if opts.ResourceOpts == nil { + opts.ResourceOpts = toNewResourceOptions(opts.OldResourceOpts) + switch opts.BindCpuOpt { + case pb.TriOpt_KEEP: + opts.ResourceOpts["keep-cpu-bind"] = newPBRawParamStr("true") + case pb.TriOpt_TRUE: + opts.ResourceOpts["cpu-bind"] = newPBRawParamStr("true") + } + } +} diff --git a/scheduler/complex/cpu.go b/scheduler/complex/cpu.go deleted file mode 100644 index 5f1395d37..000000000 --- a/scheduler/complex/cpu.go +++ /dev/null @@ -1,108 +0,0 @@ -/* -CPU 分配的核心算法 -*/ - -package complexscheduler - -import ( - "context" - "math" - "sort" - - "github.com/projecteru2/core/log" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/types" - "github.com/projecteru2/core/utils" - - "github.com/pkg/errors" -) - -func min(a, b int) int { - if a < b { - return a - } - return b -} - -func cpuPriorPlan(ctx context.Context, cpu float64, memory int64, scheduleInfos []resourcetypes.ScheduleInfo, maxShareCore, coreShare int) ([]resourcetypes.ScheduleInfo, map[string][]types.CPUMap, int, error) { - var nodeWorkload = map[string][]types.CPUMap{} - volTotal := 0 - - for p, scheduleInfo := range scheduleInfos { - // 统计全局 CPU,为非 numa 或者跨 numa 计算 - globalCPUMap := scheduleInfo.CPU - // 统计全局 Memory - globalMemCap := scheduleInfo.MemCap - // 计算每个 numa node 的分配策略 - // 得到 numa CPU 分组 - numaCPUMap := map[string]types.CPUMap{} - for cpuID, nodeID := range scheduleInfo.NUMA { - if _, ok := numaCPUMap[nodeID]; !ok { - numaCPUMap[nodeID] = types.CPUMap{} - } - cpuCount, ok := scheduleInfo.CPU[cpuID] - if !ok { - continue - } - numaCPUMap[nodeID][cpuID] = cpuCount - } - for nodeID, nodeCPUMap := range numaCPUMap { - nodeMemCap, ok := scheduleInfo.NUMAMemory[nodeID] - if !ok { - continue - } - cap, plan := calculateCPUPlan(nodeCPUMap, nodeMemCap, cpu, memory, maxShareCore, coreShare) - if cap > 0 { - if _, ok := nodeWorkload[scheduleInfo.Name]; !ok { - nodeWorkload[scheduleInfo.Name] = []types.CPUMap{} - } - volTotal += updateScheduleInfoCapacity(&scheduleInfos[p], cap) - globalMemCap -= int64(cap) * memory - for _, cpuPlan := range plan { - globalCPUMap.Sub(cpuPlan) - nodeWorkload[scheduleInfo.Name] = append(nodeWorkload[scheduleInfo.Name], cpuPlan) - } - } - log.Infof(ctx, "[cpuPriorPlan] node %s numa node %s deploy capacity %d", scheduleInfo.Name, nodeID, cap) - } - // 非 numa - // 或者是扣掉 numa 分配后剩下的资源里面 - cap, plan := calculateCPUPlan(globalCPUMap, globalMemCap, cpu, memory, maxShareCore, coreShare) - if cap > 0 { - if _, ok := nodeWorkload[scheduleInfo.Name]; !ok { - nodeWorkload[scheduleInfo.Name] = []types.CPUMap{} - } - scheduleInfos[p].Capacity += cap - volTotal += cap - nodeWorkload[scheduleInfo.Name] = append(nodeWorkload[scheduleInfo.Name], plan...) - } - log.Infof(ctx, "[cpuPriorPlan] node %s total deploy capacity %d", scheduleInfo.Name, scheduleInfos[p].Capacity) - } - - // 裁剪掉不能部署的 - sort.Slice(scheduleInfos, func(i, j int) bool { return scheduleInfos[i].Capacity < scheduleInfos[j].Capacity }) - p := sort.Search(len(scheduleInfos), func(i int) bool { return scheduleInfos[i].Capacity > 0 }) - if p == len(scheduleInfos) { - return nil, nil, 0, errors.Wrapf(types.ErrInsufficientRes, "no node remains %.2f pieces of cpu and %d bytes of memory at the same time", cpu, memory) - } - - return scheduleInfos[p:], nodeWorkload, volTotal, nil -} - -func calculateCPUPlan(CPUMap types.CPUMap, MemCap int64, cpu float64, memory int64, maxShareCore, coreShare int) (int, []types.CPUMap) { - host := newHost(CPUMap, coreShare) - plan := host.distributeOneRation(cpu, maxShareCore) - memLimit := math.MaxInt64 - if memory != 0 { - memLimit = utils.Max(int(MemCap/memory), 0) - } - cap := len(plan) // 每个node可以放的容器数 - if cap > memLimit { - plan = plan[:memLimit] - cap = memLimit - } - if cap <= 0 { - plan = nil - } - return cap, plan -} diff --git a/scheduler/complex/cpu_test.go b/scheduler/complex/cpu_test.go deleted file mode 100644 index 7691858de..000000000 --- a/scheduler/complex/cpu_test.go +++ /dev/null @@ -1,285 +0,0 @@ -package complexscheduler - -import ( - "context" - "reflect" - "testing" - - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/types" - - "github.com/docker/go-units" - "github.com/stretchr/testify/assert" -) - -func TestCPUPriorPlan(t *testing.T) { - // normal 分配 - scheduleInfos := resetscheduleInfos() - _, resultCPUPlan, total, err := cpuPriorPlan(context.TODO(), 3.0, int64(units.MiB), scheduleInfos, -1, 100) - assert.NoError(t, err) - assert.Equal(t, len(resultCPUPlan), 1) - assert.Equal(t, total, 1) - // numa 分配 - scheduleInfos = resetscheduleInfos() - _, resultCPUPlan, total, err = cpuPriorPlan(context.TODO(), 1.5, int64(units.MiB), scheduleInfos, -1, 100) - assert.NoError(t, err) - assert.Equal(t, len(resultCPUPlan), 1) - assert.Equal(t, total, 2) - r := resultCPUPlan["n1"] - for _, p := range r { - _, ok1 := p["1"] - _, ok2 := p["2"] - _, ok3 := p["3"] - _, ok4 := p["4"] - assert.True(t, (ok1 && ok3) || (ok2 && ok4)) - } - // numa and normal 分配 - scheduleInfos = resetscheduleInfos() - _, resultCPUPlan, total, err = cpuPriorPlan(context.TODO(), 1, int64(units.GiB), scheduleInfos, -1, 100) - assert.NoError(t, err) - assert.Equal(t, len(resultCPUPlan), 1) - assert.Equal(t, total, 3) -} - -func resetscheduleInfos() []resourcetypes.ScheduleInfo { - return []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{"1": 100, "2": 100, "3": 100, "4": 100}, - MemCap: 3 * int64(units.GiB), - NUMA: types.NUMA{ - "1": "node0", - "2": "node1", - "3": "node0", - "4": "node1", - }, - NUMAMemory: types.NUMAMemory{ - "node0": int64(units.GiB), - "node1": int64(units.GiB), - }, - }, - }, - } -} - -func TestCPUReallocPlan(t *testing.T) { - // shrink: 1.7->1.0 - scheduleInfo := resourcetypes.ScheduleInfo{ - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{ - "0": 0, - "1": 30, - "2": 0, - }, - }, - } - CPU := types.CPUMap{ - "0": 100, - "1": 30, - "2": 40, - } - si, remain, aff := cpuReallocPlan(scheduleInfo, 1, CPU, 100) - assert.EqualValues(t, 0, remain) - assert.True(t, reflect.DeepEqual(aff, types.CPUMap{"0": 100})) - assert.True(t, reflect.DeepEqual(si.CPU, types.CPUMap{"0": 0, "1": 60, "2": 40})) - - // shrink: 1.7->1.2 - scheduleInfo = resourcetypes.ScheduleInfo{ - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{ - "0": 0, - "1": 30, - "2": 0, - }, - }, - } - CPU = types.CPUMap{ - "0": 100, - "1": 30, - "2": 40, - } - si, remain, aff = cpuReallocPlan(scheduleInfo, 1.2, CPU, 100) - assert.EqualValues(t, 0, remain) - assert.True(t, reflect.DeepEqual(aff, types.CPUMap{"0": 100, "2": 20})) - assert.True(t, reflect.DeepEqual(si.CPU, types.CPUMap{"0": 0, "1": 60, "2": 20})) - - // expand: 1.7->2, find complement - scheduleInfo = resourcetypes.ScheduleInfo{ - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{ - "0": 0, - "1": 80, - "2": 0, - "3": 0, - }, - }, - } - CPU = types.CPUMap{ - "0": 100, - "1": 20, - "2": 40, - "3": 10, - } - si, remain, aff = cpuReallocPlan(scheduleInfo, 2, CPU, 100) - assert.EqualValues(t, 0, remain) - assert.True(t, reflect.DeepEqual(aff, types.CPUMap{"0": 100, "1": 100})) - assert.True(t, reflect.DeepEqual(si.CPU, types.CPUMap{"0": 0, "1": 0, "2": 40, "3": 10})) - - // expand: 1.7->2, lose complement - scheduleInfo = resourcetypes.ScheduleInfo{ - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{ - "0": 0, - "1": 69, - "2": 10, - }, - }, - } - CPU = types.CPUMap{ - "0": 100, - "1": 30, - "2": 40, - } - si, remain, aff = cpuReallocPlan(scheduleInfo, 2, CPU, 100) - assert.EqualValues(t, 1, remain) - assert.True(t, reflect.DeepEqual(aff, types.CPUMap{"0": 100})) - assert.True(t, reflect.DeepEqual(si.CPU, types.CPUMap{"0": 0, "1": 99, "2": 50})) -} - -func TestCPUReallocWithPriorPlan(t *testing.T) { - po, err := New(types.Config{Scheduler: types.SchedConfig{ - MaxShare: 0, - ShareBase: 100, - }}) - assert.Nil(t, err) - - // direct return after realloc plan - scheduleInfo := resourcetypes.ScheduleInfo{ - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{ - "0": 0, - "1": 70, - "2": 0, - }, - }, - } - CPU := types.CPUMap{ - "0": 100, - "1": 30, - "2": 40, - } - si, cpuPlans, total, err := po.ReselectCPUNodes(context.TODO(), scheduleInfo, CPU, 2, 0) - assert.Nil(t, err) - assert.EqualValues(t, 1, total) - assert.True(t, reflect.DeepEqual(cpuPlans, map[string][]types.CPUMap{"n1": {{"0": 100, "1": 100}}})) - assert.EqualValues(t, 1, si.Capacity) - - // realloc plan + cpu prior plan - scheduleInfo = resourcetypes.ScheduleInfo{ - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{ - "0": 100, - "1": 60, - "2": 0, - "3": 100, - "4": 100, - }, - }, - } - CPU = types.CPUMap{ - "0": 100, - "1": 30, - "2": 40, - } - si, cpuPlans, total, err = po.ReselectCPUNodes(context.TODO(), scheduleInfo, CPU, 2, 0) - assert.Nil(t, err) - assert.EqualValues(t, 3, total) - asserted := 0 - for _, plan := range cpuPlans["n1"] { - if _, ok := plan["3"]; ok { - assert.True(t, reflect.DeepEqual(plan, types.CPUMap{"0": 100, "3": 100})) - asserted++ - } else if _, ok := plan["4"]; ok { - assert.True(t, reflect.DeepEqual(plan, types.CPUMap{"0": 100, "4": 100})) - asserted++ - } else { - assert.True(t, reflect.DeepEqual(plan, types.CPUMap{"0": 200})) - asserted++ - } - } - assert.EqualValues(t, 3, asserted) - assert.EqualValues(t, 3, si.Capacity) - - // realloc plan + cpu prior error - scheduleInfo = resourcetypes.ScheduleInfo{ - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{ - "0": 0, - "1": 60, - "2": 0, - }, - }, - } - CPU = types.CPUMap{ - "0": 100, - "1": 30, - "2": 40, - } - _, _, _, err = po.ReselectCPUNodes(context.TODO(), scheduleInfo, CPU, 2, 0) - assert.EqualError(t, err, "failed to reschedule cpu: no node remains 1.00 pieces of cpu and 0 bytes of memory at the same time: not enough resource") -} - -func TestGetFullResult(t *testing.T) { - h := host{share: 100} - res := h.getFullResult(2, []resourceInfo{ - { - id: "0", - pieces: 400, - }, - { - id: "1", - pieces: 200, - }, - { - id: "2", - pieces: 400, - }, - }) - assert.EqualValues(t, 5, len(res)) - assert.ElementsMatch(t, res, []types.ResourceMap{ - {"0": 100, "1": 100}, - {"0": 100, "2": 100}, - {"0": 100, "2": 100}, - {"0": 100, "2": 100}, - {"1": 100, "2": 100}, - }) - - res = h.getFullResult(2, []resourceInfo{ - { - id: "0", - pieces: 200, - }, - { - id: "1", - pieces: 200, - }, - { - id: "2", - pieces: 200, - }, - }) - assert.EqualValues(t, 3, len(res)) - assert.ElementsMatch(t, res, []types.ResourceMap{ - {"0": 100, "1": 100}, - {"0": 100, "2": 100}, - {"1": 100, "2": 100}, - }) -} diff --git a/scheduler/complex/potassium.go b/scheduler/complex/potassium.go deleted file mode 100644 index f45577ae5..000000000 --- a/scheduler/complex/potassium.go +++ /dev/null @@ -1,457 +0,0 @@ -package complexscheduler - -import ( - "context" - "math" - "sort" - - "github.com/projecteru2/core/log" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/types" - "github.com/projecteru2/core/utils" - - "github.com/pkg/errors" -) - -// Potassium is a scheduler -type Potassium struct { - maxshare, sharebase int -} - -// New a potassium -func New(config types.Config) (*Potassium, error) { - return &Potassium{config.Scheduler.MaxShare, config.Scheduler.ShareBase}, nil -} - -// MaxIdleNode use for build -func (m *Potassium) MaxIdleNode(nodes []*types.Node) (*types.Node, error) { - if len(nodes) < 1 { - return nil, errors.WithStack(types.ErrInsufficientNodes) - } - pos := 0 - node := nodes[pos] - max := float64(node.CPU.Total())/float64(node.InitCPU.Total()) + float64(node.MemCap)/float64(node.InitMemCap) - for i, node := range nodes { - idle := float64(node.CPU.Total())/float64(node.InitCPU.Total()) + float64(node.MemCap)/float64(node.InitMemCap) - if idle > max { - pos = i - max = idle - } - } - return nodes[pos], nil -} - -// SelectStorageNodes filters nodes with enough storage -func (m *Potassium) SelectStorageNodes(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo, storage int64) ([]resourcetypes.ScheduleInfo, int, error) { - switch { - case storage < 0: - return nil, 0, errors.WithStack(types.ErrNegativeStorage) - case storage == 0: - return scheduleInfos, math.MaxInt64, nil - default: - storages := []struct { - Nodename string - Storage int64 - }{} - for _, scheduleInfo := range scheduleInfos { - storages = append(storages, struct { - Nodename string - Storage int64 - }{scheduleInfo.Name, scheduleInfo.StorageCap}) - } - log.Infof(ctx, "[SelectStorageNodes] resources: %v, need: %d", storages, storage) - } - - leng := len(scheduleInfos) - - sort.Slice(scheduleInfos, func(i, j int) bool { return scheduleInfos[i].StorageCap < scheduleInfos[j].StorageCap }) - p := sort.Search(leng, func(i int) bool { return scheduleInfos[i].StorageCap >= storage }) - if p == leng { - return nil, 0, errors.Wrapf(types.ErrInsufficientStorage, "no node remains storage more than %d bytes", storage) - } - - scheduleInfos = scheduleInfos[p:] - - total := 0 - for i := range scheduleInfos { - storCap := int(scheduleInfos[i].StorageCap / storage) - total += updateScheduleInfoCapacity(&scheduleInfos[i], storCap) - } - - return scheduleInfos, total, nil -} - -// SelectMemoryNodes filter nodes with enough memory -func (m *Potassium) SelectMemoryNodes(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo, quota float64, memory int64) ([]resourcetypes.ScheduleInfo, int, error) { - resources := []struct { - Nodename string - CPU types.CPUMap - Memory int64 - }{} - for _, scheduleInfo := range scheduleInfos { - resources = append(resources, struct { - Nodename string - CPU types.CPUMap - Memory int64 - }{scheduleInfo.Name, scheduleInfo.CPU, scheduleInfo.MemCap}) - } - log.Infof(ctx, "[SelectMemoryNodes] resources: %v, need cpu: %f, memory: %d", resources, quota, memory) - scheduleInfosLength := len(scheduleInfos) - - // 筛选出能满足 CPU 需求的 - sort.Slice(scheduleInfos, func(i, j int) bool { return len(scheduleInfos[i].CPU) < len(scheduleInfos[j].CPU) }) - p := sort.Search(scheduleInfosLength, func(i int) bool { - return float64(len(scheduleInfos[i].CPU)) >= quota - }) - // p 最大也就是 scheduleInfosLength - 1 - if p == scheduleInfosLength { - return nil, 0, errors.Wrapf(types.ErrInsufficientCPU, "no node remains cpu more than %0.2f", quota) - } - scheduleInfosLength -= p - scheduleInfos = scheduleInfos[p:] - - // 计算是否有足够的内存满足需求 - sort.Slice(scheduleInfos, func(i, j int) bool { return scheduleInfos[i].MemCap < scheduleInfos[j].MemCap }) - p = sort.Search(scheduleInfosLength, func(i int) bool { return scheduleInfos[i].MemCap >= memory }) - if p == scheduleInfosLength { - return nil, 0, errors.Wrapf(types.ErrInsufficientMEM, "no node remains memory more than %d bytes", memory) - } - scheduleInfos = scheduleInfos[p:] - - // 这里 memCap 一定是大于 memory 的所以不用判断 cap 内容 - volTotal := 0 - for i, scheduleInfo := range scheduleInfos { - capacity := math.MaxInt32 - if memory != 0 { - capacity = int(scheduleInfo.MemCap / memory) - } - volTotal += capacity - scheduleInfos[i].Capacity = capacity - } - return scheduleInfos, volTotal, nil -} - -// SelectCPUNodes select nodes with enough cpus -func (m *Potassium) SelectCPUNodes(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo, quota float64, memory int64) ([]resourcetypes.ScheduleInfo, map[string][]types.CPUMap, int, error) { - resources := []struct { - Nodename string - Memory int64 - CPU types.CPUMap - }{} - for _, scheduleInfo := range scheduleInfos { - resources = append(resources, struct { - Nodename string - Memory int64 - CPU types.CPUMap - }{scheduleInfo.Name, scheduleInfo.MemCap, scheduleInfo.CPU}) - } - log.Infof(ctx, "[SelectCPUNodes] resources %v, need cpu: %f memory: %d", resources, quota, memory) - if quota <= 0 { - return nil, nil, 0, errors.WithStack(types.ErrNegativeCPU) - } - if len(scheduleInfos) == 0 { - return nil, nil, 0, errors.WithStack(types.ErrZeroNodes) - } - - return cpuPriorPlan(ctx, quota, memory, scheduleInfos, m.maxshare, m.sharebase) -} - -// ReselectCPUNodes used for realloc one container with cpu affinity -func (m *Potassium) ReselectCPUNodes(ctx context.Context, scheduleInfo resourcetypes.ScheduleInfo, CPU types.CPUMap, quota float64, memory int64) (resourcetypes.ScheduleInfo, map[string][]types.CPUMap, int, error) { - log.Infof(ctx, "[ReselectCPUNodes] resources %v, need cpu %f, need memory %d, existing %v", - struct { - Nodename string - Memory int64 - CPU types.CPUMap - }{scheduleInfo.Name, scheduleInfo.MemCap, scheduleInfo.CPU}, - quota, memory, CPU) - var affinityPlan types.CPUMap - // remaining quota that's impossible to achieve affinity - if scheduleInfo, quota, affinityPlan = cpuReallocPlan(scheduleInfo, quota, CPU, int64(m.sharebase)); quota == 0 { - cpuPlans := map[string][]types.CPUMap{ - scheduleInfo.Name: { - affinityPlan, - }, - } - scheduleInfo.Capacity = 1 - return scheduleInfo, cpuPlans, 1, nil - } - - scheduleInfos, cpuPlans, total, err := m.SelectCPUNodes(ctx, []resourcetypes.ScheduleInfo{scheduleInfo}, quota, memory) - if err != nil { - return scheduleInfo, nil, 0, errors.Wrap(err, "failed to reschedule cpu") - } - - // add affinity plans - for i, plan := range cpuPlans[scheduleInfo.Name] { - for cpuID, pieces := range affinityPlan { - if _, ok := plan[cpuID]; ok { - cpuPlans[scheduleInfo.Name][i][cpuID] += pieces - } else { - cpuPlans[scheduleInfo.Name][i][cpuID] = pieces - } - } - } - return scheduleInfos[0], cpuPlans, total, nil -} - -func cpuReallocPlan(scheduleInfo resourcetypes.ScheduleInfo, quota float64, CPU types.CPUMap, sharebase int64) (resourcetypes.ScheduleInfo, float64, types.CPUMap) { - affinityPlan := make(types.CPUMap) - diff := int64(quota*float64(sharebase)) - CPU.Total() - // sort by pieces - cpuIDs := []string{} - for cpuID := range CPU { - cpuIDs = append(cpuIDs, cpuID) - } - sort.Slice(cpuIDs, func(i, j int) bool { return CPU[cpuIDs[i]] < CPU[cpuIDs[j]] }) - - // shrink, ensure affinity - if diff <= 0 { - affinityPlan = CPU - // prioritize fragments - for _, cpuID := range cpuIDs { - if diff == 0 { - break - } - shrink := utils.Min(affinityPlan[cpuID], -diff) - affinityPlan[cpuID] -= shrink - if affinityPlan[cpuID] == 0 { - delete(affinityPlan, cpuID) - } - diff += shrink - scheduleInfo.CPU[cpuID] += shrink - } - return scheduleInfo, float64(0), affinityPlan - } - - // expand, prioritize full cpus - needPieces := int64(quota * float64(sharebase)) - for i := len(cpuIDs) - 1; i >= 0; i-- { - cpuID := cpuIDs[i] - if needPieces == 0 { - scheduleInfo.CPU[cpuID] += CPU[cpuID] - continue - } - - // whole cpu, keep it - if CPU[cpuID] == sharebase { - affinityPlan[cpuID] = sharebase - needPieces -= sharebase - continue - } - - // fragments, try to find complement - if available := scheduleInfo.CPU[cpuID]; available == sharebase-CPU[cpuID] { - expand := utils.Min(available, needPieces) - affinityPlan[cpuID] = CPU[cpuID] + expand - scheduleInfo.CPU[cpuID] -= expand - needPieces -= sharebase - continue - } - - // else, return to cpu pools - scheduleInfo.CPU[cpuID] += CPU[cpuID] - } - - return scheduleInfo, float64(needPieces) / float64(sharebase), affinityPlan -} - -// SelectVolumeNodes calculates plans for volume request -func (m *Potassium) SelectVolumeNodes(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo, vbs types.VolumeBindings) ([]resourcetypes.ScheduleInfo, map[string][]types.VolumePlan, int, error) { - resources := []struct { - Nodename string - Volume types.VolumeMap - }{} - for _, scheduleInfo := range scheduleInfos { - resources = append(resources, struct { - Nodename string - Volume types.VolumeMap - }{scheduleInfo.Name, scheduleInfo.Volume}) - } - log.Infof(ctx, "[SelectVolumeNodes] resources %v, need volume: %v", resources, vbs.ToStringSlice(true, true)) - - var reqsNorm, reqsMono []int64 - var vbsNorm, vbsMono, vbsUnlimited types.VolumeBindings - - for _, vb := range vbs { - switch { - case vb.RequireScheduleMonopoly(): - vbsMono = append(vbsMono, vb) - reqsMono = append(reqsMono, vb.SizeInBytes) - case vb.RequireScheduleUnlimitedQuota(): - vbsUnlimited = append(vbsUnlimited, vb) - case vb.RequireSchedule(): - vbsNorm = append(vbsNorm, vb) - reqsNorm = append(reqsNorm, vb.SizeInBytes) - } - } - - if len(vbsNorm) == 0 && len(vbsMono) == 0 && len(vbsUnlimited) == 0 { - for i := range scheduleInfos { - scheduleInfos[i].Capacity = math.MaxInt64 - } - return scheduleInfos, nil, math.MaxInt64, nil - } - - volTotal := 0 - volumePlans := map[string][]types.VolumePlan{} - for idx, scheduleInfo := range scheduleInfos { - if len(scheduleInfo.Volume) == 0 { - volTotal += updateScheduleInfoCapacity(&scheduleInfos[idx], 0) - continue - } - - usedVolumeMap, unusedVolumeMap := scheduleInfo.Volume.SplitByUsed(scheduleInfo.InitVolume) - if len(reqsMono) == 0 { - usedVolumeMap.Add(unusedVolumeMap) - } - if len(reqsNorm) != 0 && len(usedVolumeMap) == 0 && len(unusedVolumeMap) != 0 { - usedVolumeMap = types.VolumeMap{} - // give out half of volumes - half, cnt, toDelete := (len(unusedVolumeMap)+1)/2, 0, []string{} - for i, v := range unusedVolumeMap { - cnt++ - if cnt > half { - break - } - toDelete = append(toDelete, i) - usedVolumeMap[i] = v - } - for _, i := range toDelete { - delete(unusedVolumeMap, i) - } - } - - capNorm, plansNorm := calculateVolumePlan(usedVolumeMap, reqsNorm) - capMono, plansMono := calculateMonopolyVolumePlan(scheduleInfo.InitVolume, unusedVolumeMap, reqsMono) - - volTotal += updateScheduleInfoCapacity(&scheduleInfos[idx], utils.Min(capNorm, capMono)) - cap := scheduleInfos[idx].Capacity - - volumePlans[scheduleInfo.Name] = make([]types.VolumePlan, cap) - for idx := range volumePlans[scheduleInfo.Name] { - volumePlans[scheduleInfo.Name][idx] = types.VolumePlan{} - } - if plansNorm != nil { - for i, plan := range plansNorm[:cap] { - volumePlans[scheduleInfo.Name][i].Merge(types.MakeVolumePlan(vbsNorm, plan)) - } - } - if plansMono != nil { - for i, plan := range plansMono[:cap] { - volumePlans[scheduleInfo.Name][i].Merge(types.MakeVolumePlan(vbsMono, plan)) - - } - } - - if len(vbsUnlimited) > 0 { - // select the device with the most capacity as unlimited plan volume - volume := types.VolumeMap{"": 0} - currentMaxAvailable := int64(0) - for vol, available := range scheduleInfo.Volume { - if available > currentMaxAvailable { - currentMaxAvailable = available - volume = types.VolumeMap{vol: 0} - } - } - - planUnlimited := types.VolumePlan{} - for _, vb := range vbsUnlimited { - planUnlimited[*vb] = volume - } - - for i := range volumePlans[scheduleInfo.Name] { - volumePlans[scheduleInfo.Name][i].Merge(planUnlimited) - } - } - } - - sort.Slice(scheduleInfos, func(i, j int) bool { return scheduleInfos[i].Capacity < scheduleInfos[j].Capacity }) - p := sort.Search(len(scheduleInfos), func(i int) bool { return scheduleInfos[i].Capacity > 0 }) - if p == len(scheduleInfos) { - return nil, nil, 0, errors.Wrapf(types.ErrInsufficientRes, "no node remains volumes for requests %+v", vbs.ToStringSlice(true, true)) - } - - return scheduleInfos[p:], volumePlans, volTotal, nil -} - -// ReselectVolumeNodes is used for realloc only -func (m *Potassium) ReselectVolumeNodes(ctx context.Context, scheduleInfo resourcetypes.ScheduleInfo, existing types.VolumePlan, vbsReq types.VolumeBindings) (resourcetypes.ScheduleInfo, map[string][]types.VolumePlan, int, error) { - log.Infof(ctx, "[ReselectVolumeNodes] resources: %v, need volume: %v, existing %v", - struct { - Nodename string - Volume types.VolumeMap - InitVolume types.VolumeMap - }{scheduleInfo.Name, scheduleInfo.Volume, scheduleInfo.InitVolume}, - vbsReq.ToStringSlice(true, true), existing.ToLiteral()) - affinityPlan := types.VolumePlan{} - needReschedule := types.VolumeBindings{} - norm, mono, unlim := distinguishVolumeBindings(vbsReq) - - // norm - normAff, normRem := distinguishAffinityVolumeBindings(norm, existing) - needReschedule = append(needReschedule, normRem...) - for _, vb := range normAff { - _, oldVM, _ := existing.FindAffinityPlan(*vb) - if scheduleInfo.Volume[oldVM.GetResourceID()] < vb.SizeInBytes { - return scheduleInfo, nil, 0, errors.Wrapf(types.ErrInsufficientVolume, "no space to expand: %+v, %+v", oldVM, vb) - } - affinityPlan.Merge(types.VolumePlan{ - *vb: types.VolumeMap{oldVM.GetResourceID(): vb.SizeInBytes}, - }) - scheduleInfo.Volume[oldVM.GetResourceID()] -= vb.SizeInBytes - } - - // mono - monoAff, _ := distinguishAffinityVolumeBindings(mono, existing) - if len(monoAff) == 0 { - // all reschedule - needReschedule = append(needReschedule, mono...) - - } else { - // all no reschedule - _, oldVM, _ := existing.FindAffinityPlan(*monoAff[0]) - monoVolume := oldVM.GetResourceID() - newVms, monoTotal, newMonoPlan := []types.VolumeMap{}, int64(0), types.VolumePlan{} - for _, vb := range mono { - monoTotal += vb.SizeInBytes - newVms = append(newVms, types.VolumeMap{monoVolume: vb.SizeInBytes}) - } - if monoTotal > scheduleInfo.InitVolume[monoVolume] { - return scheduleInfo, nil, 0, errors.Wrap(types.ErrInsufficientVolume, "no space to expand mono volumes: ") - } - newVms = proportionPlan(newVms, scheduleInfo.InitVolume[monoVolume]) - for i, vb := range mono { - newMonoPlan[*vb] = newVms[i] - } - affinityPlan.Merge(newMonoPlan) - scheduleInfo.Volume[monoVolume] = 0 - } - - // unlimit - unlimAff, unlimRem := distinguishAffinityVolumeBindings(unlim, existing) - needReschedule = append(needReschedule, unlimRem...) - unlimPlan := types.VolumePlan{} - for _, vb := range unlimAff { - _, oldVM, _ := existing.FindAffinityPlan(*vb) - unlimPlan[*vb] = oldVM - } - affinityPlan.Merge(unlimPlan) - - // schedule new volume requests - if len(needReschedule) == 0 { - scheduleInfo.Capacity = 1 - return scheduleInfo, map[string][]types.VolumePlan{scheduleInfo.Name: {affinityPlan}}, 1, nil - } - scheduleInfos, volumePlans, total, err := m.SelectVolumeNodes(ctx, []resourcetypes.ScheduleInfo{scheduleInfo}, needReschedule) - if err != nil { - return scheduleInfo, nil, 0, errors.Wrap(err, "failed to reschedule volume") - } - - // merge - for i := range volumePlans[scheduleInfo.Name] { - volumePlans[scheduleInfo.Name][i].Merge(affinityPlan) - } - - return scheduleInfos[0], volumePlans, total, nil -} diff --git a/scheduler/complex/potassium_test.go b/scheduler/complex/potassium_test.go deleted file mode 100644 index a295f7d7b..000000000 --- a/scheduler/complex/potassium_test.go +++ /dev/null @@ -1,1930 +0,0 @@ -package complexscheduler - -import ( - "context" - "errors" - "fmt" - "math" - "reflect" - "sort" - "testing" - - "github.com/projecteru2/core/resources" - "github.com/projecteru2/core/resources/cpumem" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/resources/volume" - "github.com/projecteru2/core/scheduler" - "github.com/projecteru2/core/strategy" - "github.com/projecteru2/core/types" - "github.com/projecteru2/core/utils" - - "github.com/docker/go-units" - pkgerrors "github.com/pkg/errors" - "github.com/stretchr/testify/assert" -) - -func newConfig() types.Config { - return types.Config{ - Etcd: types.EtcdConfig{ - Machines: []string{"http://127.0.0.1:2379"}, - LockPrefix: "core/_lock", - }, - Scheduler: types.SchedConfig{ - ShareBase: 10, - MaxShare: -1, - }, - } -} - -func newPotassium() (*Potassium, error) { - coreCfg := newConfig() - potassium, err := New(coreCfg) - if err != nil { - return nil, fmt.Errorf("Create Potassim error: %v", err) - } - scheduler.InitSchedulerV1(potassium) - return potassium, nil -} - -func generateNodes(nums, cores int, memory, storage int64, shares int) []resourcetypes.ScheduleInfo { - return utils.GenerateScheduleInfos(nums, cores, memory, storage, shares) -} - -func getNodesCapacity(nodes []resourcetypes.ScheduleInfo, cpu float64, shares, maxshare int) int { - var res int - var host *host - var plan []types.CPUMap - - for _, scheduleInfo := range nodes { - host = newHost(scheduleInfo.CPU, shares) - plan = host.distributeOneRation(cpu, maxshare) - res += len(plan) - } - return res -} - -func checkAvgPlan(res map[string][]types.CPUMap, minCon int, maxCon int, name string) error { - var minC int - var maxC int - var temp int - for _, v := range res { - temp = len(v) - if minC > temp || minC == 0 { - minC = temp - } - if maxC < temp { - maxC = temp - } - } - if minC != minCon || maxC != maxCon { - return fmt.Errorf("alloc plan error") - } - return nil -} - -func refreshPod(nodes []resourcetypes.ScheduleInfo, deployMap map[string]int, memory, storage int64) { - - for i := range nodes { - nodes[i].MemCap -= int64(deployMap[nodes[i].Name]) * memory - nodes[i].StorageCap -= int64(deployMap[nodes[i].Name]) * storage - } -} - -func getComplexNodes() []resourcetypes.ScheduleInfo { - return []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 2 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n1", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 7 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - "8": 10, "9": 10, "10": 10, "11": 10, - "12": 10, "13": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n2", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 6 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - "8": 10, "9": 10, "10": 10, "11": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n3", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 9 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - "8": 10, "9": 10, "10": 10, "11": 10, - "12": 10, "13": 10, "14": 10, "15": 10, - "16": 10, "17": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n4", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 4 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n5", - }, - }, - } -} - -func getEvenPlanNodes() []resourcetypes.ScheduleInfo { - return []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 4 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n1", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 5 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - "8": 10, "9": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n2", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 6 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - "8": 10, "9": 10, "10": 10, "11": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n3", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 5 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - "8": 10, "9": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n4", - }, - }, - } -} - -func getNodeMapFromscheduleInfos(scheduleInfos []resourcetypes.ScheduleInfo) map[string]*types.Node { - nodeMap := map[string]*types.Node{} - for _, scheduleInfo := range scheduleInfos { - nodeMap[scheduleInfo.Name] = &types.Node{ - NodeMeta: types.NodeMeta{ - MemCap: scheduleInfo.MemCap, - CPU: scheduleInfo.CPU, - StorageCap: scheduleInfo.StorageCap, - Name: scheduleInfo.Name, - Volume: scheduleInfo.Volume, - InitVolume: scheduleInfo.InitVolume, - }, - } - } - return nodeMap -} - -func getInfosFromscheduleInfos(scheduleInfos []resourcetypes.ScheduleInfo, planMap []resourcetypes.ResourcePlans, countMap map[string]int) (strategyInfos []strategy.Info, total int) { - if countMap == nil { - countMap = map[string]int{} - } - for _, scheduleInfo := range scheduleInfos { - capacity := math.MaxInt64 - for _, v := range planMap { - capacity = utils.Min(capacity, v.Capacity()[scheduleInfo.Name]) - } - if scheduleInfo.Capacity > 0 { - capacity = utils.Min(capacity, scheduleInfo.Capacity) - } - if capacity == math.MaxInt64 { - capacity = 0 - } - if capacity == 0 { - continue - } - total += capacity - strategyInfos = append(strategyInfos, strategy.Info{ - Nodename: scheduleInfo.Name, - Count: countMap[scheduleInfo.Name], - Capacity: capacity, - }) - } - return -} - -func newDeployOptions(need int, each bool) *types.DeployOptions { - opts := &types.DeployOptions{ - DeployStrategy: strategy.Auto, - Count: need, - } - if each { - opts.DeployStrategy = strategy.Each - } - return opts -} - -func SelectCPUNodes(k *Potassium, scheduleInfos []resourcetypes.ScheduleInfo, countMap map[string]int, quota float64, memory int64, need int, each bool) (map[string][]types.CPUMap, map[string]types.CPUMap, error) { - rrs, err := resources.MakeRequests(types.ResourceOptions{CPUQuotaLimit: quota, MemoryLimit: memory, CPUBind: true}) - if err != nil { - return nil, nil, err - } - nodeMap := getNodeMapFromscheduleInfos(scheduleInfos) - planMap, err := resources.SelectNodesByResourceRequests(context.TODO(), rrs, nodeMap) - if err != nil { - return nil, nil, err - } - - infos, total := getInfosFromscheduleInfos(scheduleInfos, planMap, countMap) - deployMap, err := strategy.Deploy(context.TODO(), newDeployOptions(need, each), infos, total) - if err != nil { - return nil, nil, err - } - result := make(map[string][]types.CPUMap) - changed := make(map[string]types.CPUMap) - for nodename, deploy := range deployMap { - for _, plan := range planMap { - if CPUPlan, ok := plan.(cpumem.ResourcePlans); ok { - result[nodename] = CPUPlan.CPUPlans[nodename][:deploy] - plan.ApplyChangesOnNode(nodeMap[nodename], utils.Range(deploy)...) - changed[nodename] = nodeMap[nodename].CPU - } - } - } - return result, changed, nil -} - -func SelectMemoryNodes(k *Potassium, scheduleInfos []resourcetypes.ScheduleInfo, countMap map[string]int, rate float64, memory int64, need int, each bool) ([]resourcetypes.ScheduleInfo, map[string]int, error) { - rrs, err := resources.MakeRequests(types.ResourceOptions{CPUQuotaLimit: rate, MemoryLimit: memory}) - if err != nil { - return nil, nil, err - } - planMap, err := resources.SelectNodesByResourceRequests(context.TODO(), rrs, getNodeMapFromscheduleInfos(scheduleInfos)) - if err != nil { - return nil, nil, err - } - - infos, total := getInfosFromscheduleInfos(scheduleInfos, planMap, countMap) - deployMap, err := strategy.Deploy(context.TODO(), newDeployOptions(need, each), infos, total) - if err != nil { - return nil, nil, err - } - return scheduleInfos, deployMap, nil -} - -func TestSelectCPUNodes(t *testing.T) { - k, _ := newPotassium() - memory := 4 * int64(units.GiB) - - _, _, err := SelectCPUNodes(k, []resourcetypes.ScheduleInfo{}, nil, 1, 1, 1, false) - assert.True(t, errors.Is(err, types.ErrZeroNodes)) - - _, _, err = SelectCPUNodes(k, []resourcetypes.ScheduleInfo{}, nil, 1, -1, 1, false) - assert.EqualError(t, err, "limit or request less than 0: bad `Memory` value") - - nodes := generateNodes(2, 2, memory, 0, 10) - _, _, err = SelectCPUNodes(k, nodes, nil, 0.5, 1, 1, false) - assert.NoError(t, err) - - _, _, err = SelectCPUNodes(k, nodes, nil, 2, 1, 3, false) - assert.True(t, errors.Is(err, types.ErrInsufficientRes)) - assert.Contains(t, err.Error(), "need: 3, available: 1") - - _, _, err = SelectCPUNodes(k, nodes, nil, 3, 1, 2, false) - assert.True(t, errors.Is(err, types.ErrInsufficientRes)) - - _, _, err = SelectCPUNodes(k, nodes, nil, 1, 1, 5, false) - assert.True(t, errors.Is(err, types.ErrInsufficientRes)) - - // new round test - nodes = generateNodes(2, 2, memory, 0, 10) - r, re, err := SelectCPUNodes(k, nodes, nil, 1, 1, 2, false) - assert.NoError(t, err) - assert.Equal(t, 2, len(r)) - assert.Equal(t, 2, len(re)) - - for nodename, cpus := range r { - assert.Contains(t, []string{"n0", "n1"}, nodename) - // assert.Equal(t, len(cpus), 1) - cpu := cpus[0] - assert.Equal(t, cpu.Total(), int64(10)) - } - - // SelectCPUNodes 里有一些副作用, 粗暴地拿一个新的来测试吧 - // 下面也是因为这个 - nodes = generateNodes(2, 2, memory, 0, 10) - r, _, err = SelectCPUNodes(k, nodes, nil, 1.3, 1, 2, false) - assert.NoError(t, err) - - for nodename, cpus := range r { - assert.Contains(t, []string{"n0", "n1"}, nodename) - assert.Equal(t, len(cpus), 1) - - cpu := cpus[0] - assert.Equal(t, cpu.Total(), int64(13)) - } -} - -func TestSelectCPUNodesWithMemoryLimit(t *testing.T) { - k, _ := newPotassium() - - _, _, _, err := k.SelectCPUNodes(context.TODO(), []resourcetypes.ScheduleInfo{}, 0, 0) - assert.Error(t, err) - - // 测试 2 个 Node,每个 CPU 10%,但是内存吃满 - nodes := generateNodes(2, 2, 1024, 0, 10) - result, _, err := SelectCPUNodes(k, nodes, nil, 0.1, 1024, 1, true) - assert.NoError(t, err) - assert.Equal(t, len(result), 2) - for _, cpus := range result { - assert.Equal(t, len(cpus), 1) - } - - // 测试 2 个 Node,内存不足 - nodes = generateNodes(2, 2, 1024, 0, 10) - _, _, err = SelectCPUNodes(k, nodes, nil, 0.1, 1025, 1, true) - assert.EqualError(t, err, "no node remains 0.10 pieces of cpu and 1025 bytes of memory at the same time: not enough resource") - - // 测试 need 超过 each node 的 capacity - nodes = generateNodes(2, 2, 1024, 0, 10) - _, _, err = SelectCPUNodes(k, nodes, nil, 0.1, 1024, 2, true) - assert.EqualError(t, err, "cannot alloc a each node plan, not enough capacity: insufficient nodes, at least 1 needed") -} - -func TestRecurrence(t *testing.T) { - // 利用相同线上数据复现线上出现的问题 - - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 0, "10": 0, "7": 0, "8": 10, "9": 10, "13": 0, "14": 0, "15": 10, "2": 10, "5": 10, "11": 0, "12": 0, "4": 0, "1": 0, "3": 10, "6": 0}, - MemCap: 12 * int64(units.GiB), - Name: "c2-node-26", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"6": 10, "10": 0, "13": 0, "14": 10, "2": 0, "7": 0, "1": 0, "11": 0, "15": 0, "8": 10, "0": 0, "3": 0, "4": 0, "5": 0, "9": 10, "12": 0}, - MemCap: 12 * int64(units.GiB), - Name: "c2-node-27", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"13": 0, "14": 0, "15": 0, "4": 10, "9": 0, "1": 0, "10": 0, "12": 10, "5": 10, "6": 10, "8": 10, "0": 0, "11": 0, "2": 10, "3": 0, "7": 0}, - MemCap: 12 * int64(units.GiB), - Name: "c2-node-28", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"15": 0, "3": 10, "0": 0, "10": 0, "13": 0, "7": 10, "8": 0, "9": 10, "12": 10, "2": 10, "4": 10, "1": 0, "11": 0, "14": 10, "5": 10, "6": 10}, - MemCap: 12 * int64(units.GiB), - Name: "c2-node-29", - }, - }, - } - - r, rp, total, err := k.SelectCPUNodes(context.TODO(), nodes, 0.5, 1) - assert.NoError(t, err) - assert.Equal(t, len(r), len(rp)) - v := 0 - for _, cpus := range rp { - v += len(cpus) - } - assert.Equal(t, v, total) -} - -func TestComplexNodes(t *testing.T) { - coreCfg := newConfig() - - k, merr := New(coreCfg) - if merr != nil { - t.Fatalf("Create Potassim error: %v", merr) - } - scheduler.InitSchedulerV1(k) - - // test1 - nodes := getComplexNodes() - res1, changed1, err := SelectCPUNodes(k, nodes, nil, 1.7, 1, 7, false) - if err != nil { - t.Fatalf("sth wrong") - } - if check := checkAvgPlan(res1, 1, 2, "res1"); check != nil { - t.Fatalf("something went wrong") - } - assert.Equal(t, len(changed1), len(res1)) - - // test2 - // SelectCPUNodes 里有一些副作用, 粗暴地拿一个新的来测试吧 - // 下面也是因为这个 - nodes = getComplexNodes() - res2, changed2, err := SelectCPUNodes(k, nodes, nil, 1.7, 1, 11, false) - if err != nil { - t.Fatalf("something went wrong") - } - if check := checkAvgPlan(res2, 2, 3, "res2"); check != nil { - t.Fatalf("something went wrong") - } - assert.Equal(t, len(changed2), len(res2)) - - // test3 - nodes = getComplexNodes() - res3, changed3, err := SelectCPUNodes(k, nodes, nil, 1.7, 1, 23, false) - assert.NoError(t, err) - if check := checkAvgPlan(res3, 2, 6, "res3"); check != nil { - t.Fatalf("something went wrong") - } - assert.Equal(t, len(changed3), len(res3)) - - // test4 - nodes = getComplexNodes() - _, _, newErr := SelectCPUNodes(k, nodes, nil, 1.6, 1, 29, false) - if newErr == nil { - t.Fatalf("how to alloc 29 workloads when you only have 28?") - } - - // test5 - nodes = getComplexNodes() - res6, _, err := SelectCPUNodes(k, nodes, nil, 1, 1, 2, true) - assert.NoError(t, err) - assert.Equal(t, len(res6), 5) -} - -func TestCPUWithMaxShareLimit(t *testing.T) { - coreCfg := newConfig() - coreCfg.Scheduler.ShareBase = 100 - coreCfg.Scheduler.MaxShare = 2 - k, err := New(coreCfg) - if err != nil { - t.Fatalf("Create Potassim error: %v", err) - } - scheduler.InitSchedulerV1(k) - - // oversell - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 100, "1": 100, "2": 100, "3": 100, "4": 100, "5": 100}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - - _, _, err = SelectCPUNodes(k, nodes, nil, 1.7, 1, 3, false) - assert.True(t, errors.Is(err, types.ErrInsufficientRes)) - assert.Contains(t, err.Error(), "available: 2") - - // test2 - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 0, "1": 0, "2": 100, "3": 100}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - _, _, err = SelectCPUNodes(k, nodes, nil, 1.2, 1, 1, false) - assert.Nil(t, err) -} - -func TestCpuOverSell(t *testing.T) { - coreCfg := newConfig() - coreCfg.Scheduler.ShareBase = 100 - k, err := New(coreCfg) - if err != nil { - t.Fatalf("Create Potassim error: %v", err) - } - scheduler.InitSchedulerV1(k) - - // oversell - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 300, "1": 300}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - - r, c, err := SelectCPUNodes(k, nodes, nil, 2, 1, 3, false) - assert.NoError(t, err) - assert.Equal(t, r["nodes1"][0]["0"], int64(100)) - assert.Equal(t, r["nodes1"][0]["1"], int64(100)) - assert.Equal(t, c["nodes1"]["0"], int64(0)) - assert.Equal(t, c["nodes1"]["1"], int64(0)) - - // oversell fragment - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 300}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - - _, _, err = SelectCPUNodes(k, nodes, nil, 0.5, 1, 6, false) - assert.NoError(t, err) - - // one core oversell - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 300}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - - _, _, err = SelectCPUNodes(k, nodes, nil, 1, 1, 2, false) - assert.NoError(t, err) - - // balance - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 100, "1": 200, "2": 300}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - _, c, err = SelectCPUNodes(k, nodes, nil, 1, 1, 2, false) - assert.NoError(t, err) - assert.Equal(t, c["nodes1"]["0"], int64(0)) - assert.Equal(t, c["nodes1"]["1"], int64(100)) - - // complex - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 50, "1": 100, "2": 300, "3": 70, "4": 200, "5": 30, "6": 230}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - _, _, err = SelectCPUNodes(k, nodes, nil, 1.7, 1, 2, false) - assert.NoError(t, err) - - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 70, "1": 100, "2": 400}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - _, c, err = SelectCPUNodes(k, nodes, nil, 1.3, 1, 4, false) - assert.NoError(t, err) - assert.Equal(t, c["nodes1"]["0"], int64(10)) - assert.Equal(t, c["nodes1"]["1"], int64(40)) - assert.Equal(t, c["nodes1"]["2"], int64(0)) -} - -func TestCPUOverSellAndStableFragmentCore(t *testing.T) { - coreCfg := newConfig() - coreCfg.Scheduler.ShareBase = 100 - k, err := New(coreCfg) - if err != nil { - t.Fatalf("Create Potassim error: %v", err) - } - scheduler.InitSchedulerV1(k) - - // oversell - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 300, "1": 300}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - - _, _, err = SelectCPUNodes(k, nodes, nil, 1.7, 1, 1, false) - assert.NoError(t, err) - - // stable fragment core - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 230, "1": 200}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - res, changed, err := SelectCPUNodes(k, nodes, nil, 1.7, 1, 1, false) - println(res) - assert.NoError(t, err) - assert.Equal(t, changed["nodes1"]["0"], int64(160)) - nodes[0].CPU = changed["nodes1"] - _, changed, err = SelectCPUNodes(k, nodes, nil, 0.3, 1, 1, false) - assert.NoError(t, err) - assert.Equal(t, changed["nodes1"]["0"], int64(130)) - assert.Equal(t, changed["nodes1"]["1"], int64(100)) - - // complex node - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 230, "1": 80, "2": 300, "3": 200}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - _, changed, err = SelectCPUNodes(k, nodes, nil, 1.7, 1, 2, false) - assert.NoError(t, err) - assert.Equal(t, changed["nodes1"]["0"], int64(160)) - assert.Equal(t, changed["nodes1"]["1"], int64(10)) - - // consume full core - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 70, "1": 50, "2": 100, "3": 100, "4": 100}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - _, changed, err = SelectCPUNodes(k, nodes, nil, 1.7, 1, 2, false) - assert.NoError(t, err) - assert.Equal(t, changed["nodes1"]["0"], int64(0)) - assert.Equal(t, changed["nodes1"]["1"], int64(50)) - - // consume less fragment core - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"0": 70, "1": 50, "2": 90}, - MemCap: 12 * int64(units.GiB), - Name: "nodes1", - }, - }, - } - _, changed, err = SelectCPUNodes(k, nodes, nil, 0.5, 1, 2, false) - assert.NoError(t, err) - assert.Equal(t, changed["nodes1"]["0"], int64(20)) - assert.Equal(t, changed["nodes1"]["1"], int64(0)) - assert.Equal(t, changed["nodes1"]["2"], int64(90)) -} - -func TestEvenPlan(t *testing.T) { - k, merr := newPotassium() - if merr != nil { - t.Fatalf("Create Potassim error: %v", merr) - } - - // nodes -- n1: 2, n2: 2 - pod1 := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ - "0": 10, "1": 10, "2": 10, "3": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "node1", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ - "0": 10, "1": 10, "2": 10, "3": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "node2", - }, - }, - } - - res1, rem1, err := SelectCPUNodes(k, pod1, nil, 1.3, 1, 2, false) - if err != nil { - t.Fatalf("sth wrong") - } - if check := checkAvgPlan(res1, 1, 1, "res1"); check != nil { - t.Fatalf("something went wrong") - } - assert.Equal(t, len(rem1), 2) - - // nodes -- n1: 4, n2: 5, n3:6, n4: 5 - pod2 := getEvenPlanNodes() - res2, rem2, _ := SelectCPUNodes(k, pod2, nil, 1.7, 1, 3, false) - if check := checkAvgPlan(res2, 1, 1, "res2"); check != nil { - t.Fatalf("something went wront") - } - assert.Equal(t, len(rem2), 3) - - pod3 := getEvenPlanNodes() - res3, rem3, _ := SelectCPUNodes(k, pod3, nil, 1.7, 1, 8, false) - if check := checkAvgPlan(res3, 2, 2, "res3"); check != nil { - t.Fatalf("something went wront") - } - assert.Equal(t, len(rem3), 4) - - pod4 := getEvenPlanNodes() - res4, rem4, _ := SelectCPUNodes(k, pod4, nil, 1.7, 1, 10, false) - if check := checkAvgPlan(res4, 2, 3, "res4"); check != nil { - t.Fatalf("something went wrong") - } - assert.Equal(t, len(rem4), 4) -} - -func TestSpecialCase(t *testing.T) { - pod := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 4 workloads - "0": 10, "1": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n1", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 5 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n2", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 6 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n3", - }, - }, - } - - k, _ := newPotassium() - res1, _, err := SelectCPUNodes(k, pod, nil, 1.7, 1, 7, false) - assert.NoError(t, err) - checkAvgPlan(res1, 1, 3, "new test 2") - - newpod := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 4 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n1", - }, - }, - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{ // 4 workloads - "0": 10, "1": 10, "2": 10, "3": 10, - "4": 10, "5": 10, "6": 10, "7": 10, - }, - MemCap: 12 * int64(units.GiB), - Name: "n2", - }, - }, - } - - res2, changed2, err := SelectCPUNodes(k, newpod, nil, 1.7, 1, 4, false) - assert.NoError(t, err) - assert.Equal(t, len(res2), len(changed2)) - checkAvgPlan(res2, 2, 2, "new test 2") -} - -func TestGetPodVol(t *testing.T) { - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - CPU: types.CPUMap{"15": 0, "3": 10, "0": 0, "10": 0, "13": 0, "7": 10, "8": 0, "9": 10, "12": 10, "2": 10, "4": 10, "1": 0, "11": 0, "14": 10, "5": 10, "6": 10}, - MemCap: 12 * int64(units.GiB), - Name: "c2-node-26", - }, - }, - } - - res := getNodesCapacity(nodes, 0.5, 10, -1) - assert.Equal(t, res, 18) - res = getNodesCapacity(nodes, 0.3, 10, -1) - assert.Equal(t, res, 27) - res = getNodesCapacity(nodes, 1.1, 10, -1) - assert.Equal(t, res, 8) -} - -// Benchmark CPU Alloc -func Benchmark_CPUAlloc(b *testing.B) { - b.StopTimer() - k, _ := newPotassium() - var cpu = 1.3 - var count = 10000 - for i := 0; i < b.N; i++ { - // 24 core, 128G memory, 10 pieces per core - hugePod := generateNodes(count, 24, 128*int64(units.GiB), 0, 10) - need := getNodesCapacity(hugePod, cpu, 10, -1) - b.StartTimer() - r, c, err := SelectCPUNodes(k, hugePod, nil, cpu, 1, need, false) - b.StopTimer() - assert.NoError(b, err) - assert.Equal(b, len(r), len(c)) - } -} - -// Benchmark Memory Alloc -func Benchmark_MemAlloc(b *testing.B) { - b.StopTimer() - k, _ := newPotassium() - var count = 10000 - // 128M per workload - var memory int64 = 1024 * 1024 * 128 - // Max vol is 128G/128M * 10000 nodes - var need = 10240000 - for i := 0; i < b.N; i++ { - // 24 core, 128G memory, 10 pieces per core - hugePod := generateNodes(count, 24, 128*int64(units.GiB), 0, 10) - b.StartTimer() - r, _, err := SelectMemoryNodes(k, hugePod, nil, 1, memory, need, false) - b.StopTimer() - assert.NoError(b, err) - assert.Equal(b, len(r), count) - } -} - -// Test SelectMemoryNodes -func TestSelectMemoryNodes(t *testing.T) { - // 2 nodes [2 workloads per node] - memory := 4 * int64(units.GiB) - pod := generateNodes(2, 2, memory, 0, 10) - k, _ := newPotassium() - // nega memory - _, _, err := SelectMemoryNodes(k, pod, nil, 1.0, -1, 4, false) - assert.Error(t, err) - - cpus := 1.0 - res, deployMap, err := SelectMemoryNodes(k, pod, nil, cpus, 512*int64(units.MiB), 4, false) - assert.NoError(t, err) - for _, node := range res { - assert.Equal(t, deployMap[node.Name], 2) - } - - // 4 nodes [1 workload on the first node] - pod = generateNodes(4, 2, memory, 0, 10) - res, deployMap, err = SelectMemoryNodes(k, pod, nil, cpus, 512*int64(units.MiB), 1, false) - assert.NoError(t, err) - assert.Equal(t, deployMap[res[0].Name], 1) - - // 4 nodes [1 workload per node] - pod = generateNodes(4, 2, memory, 0, 10) - res, deployMap, err = SelectMemoryNodes(k, pod, nil, cpus, 512*int64(units.MiB), 4, false) - assert.NoError(t, err) - for _, node := range res { - assert.Equal(t, deployMap[node.Name], 1) - } - - // 4 nodes - pod = generateNodes(4, 2, memory, 0, 10) - countMap := map[string]int{} - for i := 0; i < 4; i++ { - countMap[pod[i].Name] += i - } - res, deployMap, err = SelectMemoryNodes(k, pod, countMap, cpus, 512*int64(units.MiB), 6, false) - assert.NoError(t, err) - for i, node := range res { - assert.Equal(t, deployMap[node.Name], 3-i) - } - - pod = generateNodes(1, 2, memory, 0, 10) - _, _, err = SelectMemoryNodes(k, pod, nil, cpus, -1, 10, false) - assert.EqualError(t, err, "limit or request less than 0: bad `Memory` value") - - // test each - pod = generateNodes(4, 2, memory, 0, 10) - each := 2 - res, deployMap, _ = SelectMemoryNodes(k, pod, nil, 1000, 1024, each, true) - for i := range res { - assert.Equal(t, deployMap[res[i].Name], each) - } -} - -func TestSelectMemoryNodesNotEnough(t *testing.T) { - memory := 4 * int64(units.MiB) - // 2 nodes [memory not enough] - pod := generateNodes(2, 2, 4*int64(units.GiB), 0, 10) - k, _ := newPotassium() - _, _, err := SelectMemoryNodes(k, pod, nil, 1, 512*int64(units.MiB), 40, false) - assert.True(t, errors.Is(err, types.ErrInsufficientRes)) - assert.Contains(t, err.Error(), "need: 40, available: 16") - - // 2 nodes [memory not enough] - pod = generateNodes(2, 2, memory, 0, 10) - _, _, err = SelectMemoryNodes(k, pod, nil, 1, 5*int64(units.GiB), 1, false) - assert.Equal(t, pkgerrors.Cause(err), types.ErrInsufficientMEM) - - // 2 nodes [cpu not enough] - pod = generateNodes(2, 2, memory, 0, 10) - _, _, err = SelectMemoryNodes(k, pod, nil, 1e10, 512*int64(units.MiB), 1, false) - assert.Equal(t, pkgerrors.Cause(err), types.ErrInsufficientCPU) -} - -func TestSelectMemoryNodesSequence(t *testing.T) { - pod := generateNodes(2, 2, 4*int64(units.GiB), 0, 10) - k, _ := newPotassium() - cpu := 1.0 - mem := 512 * int64(units.MiB) - res, deployMap, err := SelectMemoryNodes(k, pod, nil, cpu, mem, 1, false) - assert.NoError(t, err) - for _, node := range res { - if node.Name == "node0" { - assert.Equal(t, deployMap[node.Name], 1) - } - } - - refreshPod(res, deployMap, mem, 0) - res, deployMap, err = SelectMemoryNodes(k, res, nil, cpu, mem, 1, false) - assert.NoError(t, err) - for _, node := range res { - if node.Name == "node1" { - assert.Equal(t, deployMap[node.Name], 1) - } - } - - refreshPod(res, deployMap, mem, 0) - res, deployMap, err = SelectMemoryNodes(k, res, nil, cpu, mem, 4, false) - assert.NoError(t, err) - assert.Equal(t, deployMap[res[0].Name], 2) - assert.Equal(t, deployMap[res[1].Name], 2) - - refreshPod(res, deployMap, mem, 0) - res, deployMap, err = SelectMemoryNodes(k, res, nil, cpu, mem, 3, false) - assert.NoError(t, err) - assert.Equal(t, deployMap[res[0].Name]+deployMap[res[1].Name], 3) - assert.Equal(t, deployMap[res[0].Name]-deployMap[res[1].Name], 1) - - refreshPod(res, deployMap, mem, 0) - _, _, err = SelectMemoryNodes(k, res, nil, cpu, mem, 40, false) - assert.True(t, errors.Is(err, types.ErrInsufficientRes)) - assert.Contains(t, err.Error(), "need: 40, available: 7") - - // new round - pod = generateNodes(2, 2, 4*int64(units.GiB), 0, 10) - res, deployMap, err = SelectMemoryNodes(k, pod, nil, cpu, mem, 1, false) - assert.NoError(t, err) - for _, node := range res { - if node.Name == "node0" { - assert.Equal(t, deployMap[node.Name], 1) - } - } - refreshPod(res, deployMap, mem, 0) - res, deployMap, err = SelectMemoryNodes(k, res, nil, cpu, mem, 2, false) - assert.NoError(t, err) - for _, node := range res { - if node.Name == "node1" { - assert.Equal(t, deployMap[node.Name], 2) - } - } - refreshPod(res, deployMap, mem, 0) - res, deployMap, err = SelectMemoryNodes(k, res, nil, cpu, mem, 5, false) - assert.NoError(t, err) - finalCounts := []int{deployMap[res[0].Name], deployMap[res[1].Name]} - sort.Ints(finalCounts) - assert.ElementsMatch(t, []int{2, 3}, finalCounts) - -} - -func TestSelectMemoryNodesGiven(t *testing.T) { - pod := generateNodes(4, 2, 4*int64(units.GiB), 0, 10) - countMap := map[string]int{} - for i := 0; i < 3; i++ { - countMap[pod[i].Name]++ - - } - - k, _ := newPotassium() - _, deployMap, err := SelectMemoryNodes(k, pod, countMap, 1.0, 512*int64(units.MiB), 2, false) - assert.NoError(t, err) - finalCounts := []int{} - for _, node := range pod { - finalCounts = append(finalCounts, countMap[node.Name]+deployMap[node.Name]) - } - sort.Ints(finalCounts) - assert.ElementsMatch(t, []int{1, 1, 1, 2}, finalCounts) - -} - -func TestMaxIdleNode(t *testing.T) { - n1 := &types.Node{ - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{"0": 20}, - InitCPU: types.CPUMap{"0": 100}, - MemCap: 30, - InitMemCap: 100, - }, - } - n2 := &types.Node{ - NodeMeta: types.NodeMeta{ - Name: "n1", - CPU: types.CPUMap{"0": 30}, - InitCPU: types.CPUMap{"0": 100}, - MemCap: 10, - InitMemCap: 100, - }, - } - k, _ := newPotassium() - _, err := k.MaxIdleNode([]*types.Node{}) - assert.Error(t, err) - node, err := k.MaxIdleNode([]*types.Node{n1, n2}) - assert.NoError(t, err) - assert.Equal(t, node.Name, n2.Name) -} - -func TestSelectStorageNodesMultipleDeployedPerNode(t *testing.T) { - k, _ := newPotassium() - emptyNode := []resourcetypes.ScheduleInfo{} - _, r, err := k.SelectStorageNodes(context.TODO(), emptyNode, -1) - assert.Zero(t, r) - assert.Error(t, err) - _, r, err = k.SelectStorageNodes(context.TODO(), emptyNode, 0) - assert.Equal(t, r, math.MaxInt64) - assert.NoError(t, err) - scheduleInfos := generateNodes(2, 2, 4*int64(units.GiB), 8*int64(units.GiB), 10) - scheduleInfos, total, err := k.SelectMemoryNodes(context.TODO(), scheduleInfos, 1.0, int64(units.GiB)) - assert.NoError(t, err) - assert.Equal(t, 8, total) - assert.Equal(t, 2, len(scheduleInfos)) - assert.Equal(t, 4, scheduleInfos[0].Capacity) - assert.Equal(t, 4, scheduleInfos[1].Capacity) - - res, deployMap, err := SelectStorageNodes(k, scheduleInfos, nil, int64(units.GiB), 4, false) - assert.NoError(t, err) - assert.Equal(t, 2, len(res)) - assert.Equal(t, 2, deployMap[res[0].Name]) - assert.Equal(t, 2, deployMap[res[1].Name]) - assert.Equal(t, 2, res[0].Capacity) - assert.Equal(t, 2, res[1].Capacity) -} - -func TestSelectStorageNodesDeployedOnFirstNode(t *testing.T) { - k, _ := newPotassium() - scheduleInfos := generateNodes(2, 2, 4*int64(units.GiB), int64(units.GiB), 10) - scheduleInfos, total, err := k.SelectMemoryNodes(context.TODO(), scheduleInfos, 1.0, int64(units.GiB)) - assert.NoError(t, err) - assert.Equal(t, 8, total) - assert.Equal(t, 2, len(scheduleInfos)) - assert.Equal(t, 4, scheduleInfos[0].Capacity) - assert.Equal(t, 4, scheduleInfos[1].Capacity) - - res, deployMap, err := SelectStorageNodes(k, scheduleInfos, nil, int64(units.MiB), 1, false) - assert.NoError(t, err) - assert.Equal(t, 2, len(res)) - assert.Equal(t, 1, deployMap[res[0].Name]) - assert.Equal(t, 0, deployMap[res[1].Name]) - assert.Equal(t, 3, res[0].Capacity) - assert.Equal(t, 4, res[1].Capacity) -} - -func TestSelectStorageNodesOneDeployedPerNode(t *testing.T) { - k, _ := newPotassium() - scheduleInfos := generateNodes(4, 2, 4*int64(units.GiB), int64(units.GiB), 10) - scheduleInfos, total, err := k.SelectMemoryNodes(context.TODO(), scheduleInfos, 1.0, int64(units.GiB)) - assert.NoError(t, err) - assert.Equal(t, 16, total) - assert.Equal(t, 4, len(scheduleInfos)) - assert.Equal(t, 4, scheduleInfos[0].Capacity) - assert.Equal(t, 4, scheduleInfos[1].Capacity) - - res, deployMap, err := SelectStorageNodes(k, scheduleInfos, nil, int64(units.MiB), 4, false) - assert.NoError(t, err) - assert.Equal(t, 4, len(res)) - for _, node := range res { - assert.Equal(t, 1, deployMap[node.Name]) - assert.Equal(t, 3, node.Capacity) - } -} - -func TestSelectStorageNodesWithPreOccupied(t *testing.T) { - k, _ := newPotassium() - scheduleInfos := generateNodes(4, 2, 4*int64(units.GiB), int64(units.GiB), 10) - // Set occupied count - countMap := map[string]int{} - for i := 0; i < 4; i++ { - countMap[scheduleInfos[i].Name] += i - } - scheduleInfos, total, err := k.SelectMemoryNodes(context.TODO(), scheduleInfos, 1.0, 512*int64(units.MiB)) - assert.NoError(t, err) - assert.Equal(t, 32, total) - assert.Equal(t, 4, len(scheduleInfos)) - for _, node := range scheduleInfos { - assert.Equal(t, 8, node.Capacity) - } - - res, deployMap, err := SelectStorageNodes(k, scheduleInfos, countMap, int64(units.MiB), 6, false) - assert.NoError(t, err) - assert.Equal(t, 4, len(res)) - for i, node := range res { - assert.Equal(t, 5+i, node.Capacity) - assert.Equal(t, 3-i, deployMap[node.Name]) - } -} - -func TestSelectStorageNodesAllocEachDivition(t *testing.T) { - k, _ := newPotassium() - scheduleInfos := generateNodes(4, 2, 4*int64(units.GiB), int64(units.GiB), 10) - scheduleInfos, total, err := k.SelectMemoryNodes(context.TODO(), scheduleInfos, 1.0, int64(units.GiB)) - assert.NoError(t, err) - assert.Equal(t, 16, total) - assert.Equal(t, 4, len(scheduleInfos)) - for _, node := range scheduleInfos { - assert.Equal(t, 4, node.Capacity) - } - - res, deployMap, err := SelectStorageNodes(k, scheduleInfos, nil, int64(units.MiB), 2, true) - assert.NoError(t, err) - assert.Equal(t, 4, len(res)) - for _, node := range res { - assert.Equal(t, 2, deployMap[node.Name]) - assert.Equal(t, 2, node.Capacity) - } -} - -func TestSelectStorageNodesCapacityLessThanMemory(t *testing.T) { - k, _ := newPotassium() - scheduleInfos := generateNodes(2, 2, 4*int64(units.GiB), int64(units.GiB), 10) - scheduleInfos, total, err := k.SelectMemoryNodes(context.TODO(), scheduleInfos, 1.0, int64(units.GiB)) - assert.NoError(t, err) - assert.Equal(t, 8, total) - assert.Equal(t, 2, len(scheduleInfos)) - assert.Equal(t, 4, scheduleInfos[0].Capacity) - assert.Equal(t, 4, scheduleInfos[1].Capacity) - - res, deployMap, err := SelectStorageNodes(k, scheduleInfos, nil, int64(units.GiB), 2, false) - assert.NoError(t, err) - assert.Equal(t, 2, len(res)) - assert.Equal(t, 1, deployMap[res[0].Name]) - assert.Equal(t, 1, deployMap[res[1].Name]) - assert.Equal(t, 0, res[0].Capacity) - assert.Equal(t, 0, res[1].Capacity) -} - -func TestSelectStorageNodesNotEnough(t *testing.T) { - k, _ := newPotassium() - scheduleInfos := generateNodes(1, 2, 4*int64(units.GiB), int64(units.MiB), 10) - scheduleInfos, total, err := k.SelectMemoryNodes(context.TODO(), scheduleInfos, 1.0, int64(units.GiB)) - assert.NoError(t, err) - assert.Equal(t, 4, total) - assert.Equal(t, 1, len(scheduleInfos)) - assert.Equal(t, 4, scheduleInfos[0].Capacity) - - res, _, err := SelectStorageNodes(k, scheduleInfos, nil, int64(units.GiB), 1, false) - assert.Equal(t, types.ErrInsufficientStorage, pkgerrors.Cause(err)) - assert.Nil(t, res) -} - -func TestSelectStorageNodesSequence(t *testing.T) { - k, _ := newPotassium() - scheduleInfos := generateNodes(2, 4, 8*int64(units.GiB), 2*int64(units.GiB), 10) - mem := 512 * int64(units.MiB) - scheduleInfos, total, err := k.SelectMemoryNodes(context.TODO(), scheduleInfos, 1.0, mem) - assert.NoError(t, err) - assert.Equal(t, 32, total) - assert.Equal(t, 2, len(scheduleInfos)) - assert.Equal(t, 16, scheduleInfos[0].Capacity) - assert.Equal(t, 16, scheduleInfos[1].Capacity) - - stor := int64(units.GiB) - res, deployMap, err := SelectStorageNodes(k, scheduleInfos, nil, stor, 1, false) - assert.NoError(t, err) - assert.Equal(t, 2, len(res)) - assert.Equal(t, 1, res[0].Capacity) - assert.Equal(t, 2, res[1].Capacity) - counts := []int{deployMap[scheduleInfos[0].Name], deployMap[scheduleInfos[1].Name]} - sort.Ints(counts) - assert.ElementsMatch(t, []int{0, 1}, counts) - - refreshPod(res, deployMap, mem, stor) - countMap := map[string]int{ - res[0].Name: 1, - res[1].Name: 0, - } - - res, total, err = k.SelectMemoryNodes(context.TODO(), res, 1.0, mem) - assert.NoError(t, err) - assert.Equal(t, 31, total) - assert.Equal(t, 2, len(res)) - assert.Equal(t, 15, res[0].Capacity) - assert.Equal(t, 16, res[1].Capacity) - lesserResourceNodeName := res[0].Name - - res, deployMap, err = SelectStorageNodes(k, res, countMap, int64(units.GiB), 2, false) - assert.NoError(t, err) - assert.Equal(t, 2, len(res)) - - getLess := func(nodes []resourcetypes.ScheduleInfo) (lesser int, greater int) { - if res[0].Name == lesserResourceNodeName { - greater = 1 - } else { - lesser = 1 - } - return - } - i, j := getLess(res) - getFinalCounts := func(deployMap, countMap map[string]int) (counts []int) { - for name, d := range deployMap { - counts = append(counts, d+countMap[name]) - } - sort.Ints(counts) - return - } - assert.ElementsMatch(t, []int{1, 2}, getFinalCounts(deployMap, countMap)) - - refreshPod(res, deployMap, mem, stor) - countMap = map[string]int{ - res[i].Name: 1, - res[j].Name: 2, - } - - res, total, err = k.SelectMemoryNodes(context.TODO(), res, 1.0, mem) - assert.NoError(t, err) - assert.Equal(t, 29, total) - assert.Equal(t, 2, len(res)) - assert.Equal(t, 14, res[0].Capacity) - assert.Equal(t, 15, res[1].Capacity) -} - -func SelectStorageNodes(k *Potassium, scheduleInfos []resourcetypes.ScheduleInfo, countMap map[string]int, storage int64, need int, each bool) ([]resourcetypes.ScheduleInfo, map[string]int, error) { - rrs, err := resources.MakeRequests(types.ResourceOptions{StorageLimit: storage}) - if err != nil { - return nil, nil, err - } - planMap, err := resources.SelectNodesByResourceRequests(context.TODO(), rrs, getNodeMapFromscheduleInfos(scheduleInfos)) - if err != nil { - return nil, nil, err - } - - strategyInfos, total := getInfosFromscheduleInfos(scheduleInfos, planMap, countMap) - deployMap, err := strategy.Deploy(context.TODO(), newDeployOptions(need, each), strategyInfos, total) - if err != nil { - return nil, nil, err - } - for i, scheduleInfo := range scheduleInfos { - for _, si := range strategyInfos { - if si.Nodename == scheduleInfo.Name { - scheduleInfos[i].Capacity = si.Capacity - deployMap[si.Nodename] - } - } - } - return scheduleInfos, deployMap, nil -} - -func SelectVolumeNodes(k *Potassium, scheduleInfos []resourcetypes.ScheduleInfo, countMap map[string]int, volumes []string, need int, each bool) (map[string][]types.VolumePlan, map[string]types.VolumeMap, error) { - rrs, err := resources.MakeRequests(types.ResourceOptions{VolumeLimit: types.MustToVolumeBindings(volumes)}) - if err != nil { - return nil, nil, err - } - nodeMap := getNodeMapFromscheduleInfos(scheduleInfos) - planMap, err := resources.SelectNodesByResourceRequests(context.TODO(), rrs, nodeMap) - if err != nil { - return nil, nil, err - } - - infos, total := getInfosFromscheduleInfos(scheduleInfos, planMap, countMap) - deployMap, err := strategy.Deploy(context.TODO(), newDeployOptions(need, each), infos, total) - if err != nil { - return nil, nil, err - } - result := make(map[string][]types.VolumePlan) - changed := make(map[string]types.VolumeMap) - for nodename, deploy := range deployMap { - for _, plan := range planMap { - if volumePlan, ok := plan.(volume.ResourcePlans); ok { - result[nodename] = volumePlan.GetPlan(nodename) - plan.ApplyChangesOnNode(nodeMap[nodename], utils.Range(deploy)...) - changed[nodename] = nodeMap[nodename].Volume - } - } - } - return result, changed, nil -} - -func TestSelectVolumeNodesNonAuto(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 1024, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{ - "/tmp:/tmp:rw:2048", - "/var/log:/var/log:ro", - "/data0:/data:rw", - "/data0:/data", - } - res, changed, err := SelectVolumeNodes(k, nodes, nil, volumes, 2, true) - assert.NoError(t, err) - assert.Equal(t, len(res["0"]), 0) - assert.Equal(t, changed["node1"]["/data0"], int64(0)) -} - -func TestSelectVolumeNodesAutoInsufficient(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 1024, - "/data1": 2048, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{"AUTO:/data:rw:2049"} - _, _, err := SelectVolumeNodes(k, nodes, nil, volumes, 1, true) - assert.True(t, errors.Is(err, types.ErrInsufficientRes)) - - volumes = []string{"AUTO:/data:rw:1024", "AUTO:/dir:rw:1024"} - _, _, err = SelectVolumeNodes(k, nodes, nil, volumes, 2, true) - assert.Contains(t, err.Error(), "not enough capacity") -} - -func TestSelectVolumeNodesAutoSingle(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 1024, - "/data1": 2048, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{"AUTO:/data:rw:70"} - res, changed, err := SelectVolumeNodes(k, nodes, nil, volumes, 43, true) - assert.Nil(t, err) - assert.Equal(t, len(res["0"]), 43) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding("AUTO:/data:rw:70")], types.VolumeMap{"/data0": 70}) - assert.Equal(t, changed["0"], types.VolumeMap{"/data0": 44, "/data1": 18}) -} - -func TestSelectVolumeNodesAutoDouble(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 1024, - "/data1": 1025, - }, - StorageCap: units.GiB, - }, - }, - { - NodeMeta: types.NodeMeta{ - Name: "1", - Volume: types.VolumeMap{ - "/data0": 2048, - "/data1": 2049, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{"AUTO:/data:rw:20", "AUTO:/dir:rw:200"} - res, changed, err := SelectVolumeNodes(k, nodes, nil, volumes, 5, true) - assert.Nil(t, err) - assert.Equal(t, res["0"][4][types.MustToVolumeBinding("AUTO:/data:rw:20")], types.VolumeMap{"/data0": 20}) - assert.Equal(t, res["0"][4][types.MustToVolumeBinding("AUTO:/dir:rw:200")], types.VolumeMap{"/data1": 200}) - assert.Equal(t, res["1"][4][types.MustToVolumeBinding("AUTO:/data:rw:20")], types.VolumeMap{"/data0": 20}) - assert.Equal(t, res["1"][4][types.MustToVolumeBinding("AUTO:/dir:rw:200")], types.VolumeMap{"/data0": 200}) - assert.Equal(t, changed["0"], types.VolumeMap{"/data0": 124, "/data1": 825}) - assert.Equal(t, changed["1"], types.VolumeMap{"/data0": 948, "/data1": 2049}) -} - -func TestSelectVolumeNodesAutoTriple(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data1": 1218, - "/data2": 1219, - "/data0": 2000, - }, - StorageCap: units.GiB, - }, - }, - { - NodeMeta: types.NodeMeta{ - Name: "1", - Volume: types.VolumeMap{ - "/data1": 100, - "/data2": 10, - "/data3": 2110, - }, - StorageCap: units.GiB, - }, - }, - { - NodeMeta: types.NodeMeta{ - Name: "2", - Volume: types.VolumeMap{ - "/data2": 1001, - "/data3": 1000, - "/data4": 1002, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{ - "AUTO:/data0:rw:1000", - "AUTO:/data1:rw:10", - "AUTO:/data2:rw:100", - } - - res, changed, err := SelectVolumeNodes(k, nodes, nil, volumes, 2, true) - assert.Nil(t, err) - assert.Equal(t, res["0"][1][types.MustToVolumeBinding("AUTO:/data0:rw:1000")], types.VolumeMap{"/data2": 1000}) - assert.Equal(t, res["0"][1][types.MustToVolumeBinding("AUTO:/data1:rw:10")], types.VolumeMap{"/data2": 10}) - assert.Equal(t, res["0"][1][types.MustToVolumeBinding("AUTO:/data2:rw:100")], types.VolumeMap{"/data1": 100}) - - assert.Equal(t, res["1"][0][types.MustToVolumeBinding("AUTO:/data0:rw:1000")], types.VolumeMap{"/data3": 1000}) - assert.Equal(t, res["1"][0][types.MustToVolumeBinding("AUTO:/data1:rw:10")], types.VolumeMap{"/data2": 10}) - assert.Equal(t, res["1"][0][types.MustToVolumeBinding("AUTO:/data2:rw:100")], types.VolumeMap{"/data1": 100}) - - assert.Equal(t, res["2"][1][types.MustToVolumeBinding("AUTO:/data0:rw:1000")], types.VolumeMap{"/data4": 1000}) - assert.Equal(t, res["2"][1][types.MustToVolumeBinding("AUTO:/data1:rw:10")], types.VolumeMap{"/data2": 10}) - assert.Equal(t, res["2"][1][types.MustToVolumeBinding("AUTO:/data2:rw:100")], types.VolumeMap{"/data2": 100}) - - assert.Equal(t, changed["0"], types.VolumeMap{"/data1": 8, "/data2": 209, "/data0": 2000}) - assert.Equal(t, changed["1"], types.VolumeMap{"/data1": 0, "/data2": 0, "/data3": 0}) - assert.Equal(t, changed["2"], types.VolumeMap{"/data2": 781, "/data3": 0, "/data4": 2}) -} - -func TestSelectMonopoly(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 2000, - "/data2": 2000, - }, - InitVolume: types.VolumeMap{ - "/data0": 2001, - "/data2": 2000, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{"AUTO:/data:rwm:997"} - res, changed, err := SelectVolumeNodes(k, nodes, nil, volumes, 1, true) - - assert.Nil(t, err) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding("AUTO:/data:rwm:997")], types.VolumeMap{"/data2": 2000}) - assert.Equal(t, changed["0"], types.VolumeMap{"/data0": 2000, "/data2": 0}) - -} - -func TestSelectMultipleMonopoly(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 2000, - "/data2": 2000, - "/data3": 3000, - }, - InitVolume: types.VolumeMap{ - "/data0": 2000, - "/data2": 2001, - "/data3": 3000, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{"AUTO:/data:rom:100", "AUTO:/data1:rom:200"} - res, changed, err := SelectVolumeNodes(k, nodes, nil, volumes, 2, true) - - assert.Nil(t, err) - assert.Equal(t, len(res["0"]), 2) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding(volumes[0])], types.VolumeMap{"/data0": 666}) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding(volumes[1])], types.VolumeMap{"/data0": 1333}) - assert.Equal(t, res["0"][1][types.MustToVolumeBinding(volumes[0])], types.VolumeMap{"/data3": 1000}) - assert.Equal(t, res["0"][1][types.MustToVolumeBinding(volumes[1])], types.VolumeMap{"/data3": 2000}) - assert.Equal(t, changed["0"], types.VolumeMap{"/data0": 1, "/data2": 2000, "/data3": 0}) -} - -func TestSelectHyperMonopoly(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 2000, - "/data2": 2000, - }, - InitVolume: types.VolumeMap{ - "/data0": 2000, - "/data2": 2001, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{ - "AUTO:/data:rom:100", "AUTO:/data1:rmw:200", "AUTO:/data2:m:300", - "AUTO:/data3:ro:100", "AUTO:/data4:rw:400", - } - res, changed, err := SelectVolumeNodes(k, nodes, nil, volumes, 1, true) - - assert.Nil(t, err) - assert.Equal(t, len(res["0"]), 1) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding(volumes[0])], types.VolumeMap{"/data0": 333}) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding(volumes[1])], types.VolumeMap{"/data0": 666}) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding(volumes[2])], types.VolumeMap{"/data0": 1000}) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding(volumes[3])], types.VolumeMap{"/data2": 100}) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding(volumes[4])], types.VolumeMap{"/data2": 400}) - assert.Equal(t, changed["0"], types.VolumeMap{"/data0": 1, "/data2": 1500}) -} - -func TestSelectMonopolyOnMultipleNodes(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 2000, - "/data1": 2000, - }, - InitVolume: types.VolumeMap{ - "/data0": 2001, - "/data1": 2000, - }, - StorageCap: units.GiB, - }, - }, - { - NodeMeta: types.NodeMeta{ - Name: "1", - Volume: types.VolumeMap{ - "/data0": 2000, - "/data1": 2000, - }, - InitVolume: types.VolumeMap{ - "/data0": 2000, - "/data1": 2001, - }, - StorageCap: units.GiB, - }, - }, - { - NodeMeta: types.NodeMeta{ - Name: "2", - }, - }, - } - - volumes := []string{"AUTO:/data:rom:100", "AUTO:/data1:wrm:300"} - res, changed, err := SelectVolumeNodes(k, nodes, nil, volumes, 1, true) - - assert.Nil(t, err) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding(volumes[0])], types.VolumeMap{"/data1": 500}) - assert.Equal(t, res["0"][0][types.MustToVolumeBinding(volumes[1])], types.VolumeMap{"/data1": 1500}) - assert.Equal(t, res["1"][0][types.MustToVolumeBinding(volumes[0])], types.VolumeMap{"/data0": 500}) - assert.Equal(t, res["1"][0][types.MustToVolumeBinding(volumes[1])], types.VolumeMap{"/data0": 1500}) - assert.Equal(t, changed["0"], types.VolumeMap{"/data0": 2000, "/data1": 0}) - assert.Equal(t, changed["1"], types.VolumeMap{"/data0": 0, "/data1": 2000}) -} - -func TestSelectMonopolyInsufficient(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 2000, - }, - InitVolume: types.VolumeMap{ - "/data0": 2001, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{"AUTO:/data:m:1"} - _, _, err := SelectVolumeNodes(k, nodes, nil, volumes, 1, true) - assert.True(t, errors.Is(err, types.ErrInsufficientRes)) - - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data0": 2000, - "/data1": 2000, - }, - InitVolume: types.VolumeMap{ - "/data0": 2000, - "/data1": 2001, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes = []string{"/AUTO:/data:m:200"} - _, _, err = SelectVolumeNodes(k, nodes, nil, volumes, 2, true) - assert.True(t, errors.Is(err, types.ErrInsufficientCap)) -} - -func TestSelectUnlimited(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "0", - Volume: types.VolumeMap{ - "/data1": 1218, - "/data2": 1219, - "/data0": 2000, - }, - StorageCap: units.GiB, - }, - }, - { - NodeMeta: types.NodeMeta{ - Name: "1", - Volume: types.VolumeMap{ - "/data1": 100, - "/data2": 10, - "/data3": 2110, - }, - StorageCap: units.GiB, - }, - }, - { - NodeMeta: types.NodeMeta{ - Name: "2", - Volume: types.VolumeMap{ - "/data2": 1001, - "/data3": 1000, - "/data4": 1002, - }, - StorageCap: units.GiB, - }, - }, - } - - volumes := []string{ - "AUTO:/data0:rw:1000", - "AUTO:/data1:rw:10", - "AUTO:/data2:rw:100", - "AUTO:/data3:rw:0", - "AUTO:/data4:rw:0", - } - - res, _, _ := SelectVolumeNodes(k, nodes, nil, volumes, 2, true) - - assert.Equal(t, res["0"][1][types.MustToVolumeBinding("AUTO:/data0:rw:1000")], types.VolumeMap{"/data2": 1000}) - assert.Equal(t, res["0"][1][types.MustToVolumeBinding("AUTO:/data1:rw:10")], types.VolumeMap{"/data2": 10}) - assert.Equal(t, res["0"][1][types.MustToVolumeBinding("AUTO:/data2:rw:100")], types.VolumeMap{"/data1": 100}) - assert.Equal(t, res["0"][1][types.MustToVolumeBinding("AUTO:/data3:rw:0")], types.VolumeMap{"/data0": 0}) - assert.Equal(t, res["0"][1][types.MustToVolumeBinding("AUTO:/data4:rw:0")], types.VolumeMap{"/data0": 0}) - - assert.Equal(t, res["1"][0][types.MustToVolumeBinding("AUTO:/data0:rw:1000")], types.VolumeMap{"/data3": 1000}) - assert.Equal(t, res["1"][0][types.MustToVolumeBinding("AUTO:/data1:rw:10")], types.VolumeMap{"/data2": 10}) - assert.Equal(t, res["1"][0][types.MustToVolumeBinding("AUTO:/data2:rw:100")], types.VolumeMap{"/data1": 100}) - assert.Equal(t, res["1"][0][types.MustToVolumeBinding("AUTO:/data3:rw:0")], types.VolumeMap{"/data3": 0}) - assert.Equal(t, res["1"][0][types.MustToVolumeBinding("AUTO:/data4:rw:0")], types.VolumeMap{"/data3": 0}) - - assert.Equal(t, res["2"][1][types.MustToVolumeBinding("AUTO:/data0:rw:1000")], types.VolumeMap{"/data4": 1000}) - assert.Equal(t, res["2"][1][types.MustToVolumeBinding("AUTO:/data1:rw:10")], types.VolumeMap{"/data2": 10}) - assert.Equal(t, res["2"][1][types.MustToVolumeBinding("AUTO:/data2:rw:100")], types.VolumeMap{"/data2": 100}) - assert.Equal(t, res["2"][1][types.MustToVolumeBinding("AUTO:/data3:rw:0")], types.VolumeMap{"/data4": 0}) - assert.Equal(t, res["2"][1][types.MustToVolumeBinding("AUTO:/data4:rw:0")], types.VolumeMap{"/data4": 0}) - -} - -func TestSelectVolumeNormAndMono(t *testing.T) { - k, _ := newPotassium() - - nodes := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "n0", - Volume: types.VolumeMap{ - "/data0": 1000, - "/data1": 2000, - }, - InitVolume: types.VolumeMap{ - "/data0": 1000, - "/data1": 2000, - }, - StorageCap: units.GiB, - }, - }, - } - - req := []string{ - "AUTO:/data0:rw:50", - "AUTO:/data1:rwm:50", - } - - res, _, err := SelectVolumeNodes(k, nodes, nil, req, 1, false) - assert.Nil(t, err) - assert.EqualValues(t, 1, len(res["n0"])) - assert.True(t, reflect.DeepEqual(res["n0"][0], types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data0:rw:50": { - "/data0": 50, - }, - "AUTO:/data1:rwm:50": { - "/data1": 2000, - }, - })) || reflect.DeepEqual(res["n0"][0], types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data0:rw:50": { - "/data1": 50, - }, - "AUTO:/data1:rwm:50": { - "/data0": 1000, - }, - }))) - - // round 2 - nodes = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "n0", - Volume: types.VolumeMap{ - "/data0": 1000, - "/data1": 2000, - "/data2": 1000, - "/data3": 2000, - }, - InitVolume: types.VolumeMap{ - "/data0": 1000, - "/data1": 2000, - "/data2": 1000, - "/data3": 2000, - }, - StorageCap: units.GiB, - }, - }, - } - - req = []string{ - "AUTO:/data0:rw:50", - "AUTO:/data1:rwm:50", - } - - res, _, err = SelectVolumeNodes(k, nodes, nil, req, 2, false) - assert.Nil(t, err) - assert.EqualValues(t, 2, len(res["n0"])) -} - -func TestSelectFromNegateResources(t *testing.T) { - k, _ := newPotassium() - - // memory - scheduleInfos := []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "n0", - MemCap: -1000, - InitMemCap: 1000, - }, - }, - } - _, total, err := k.SelectMemoryNodes(context.Background(), scheduleInfos, 0, 100) - assert.EqualValues(t, 0, total) - assert.EqualError(t, err, "no node remains memory more than 100 bytes: cannot alloc a plan, not enough memory") - - // cpu - scheduleInfos = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "n0", - CPU: types.CPUMap{"0": -100, "1": -2000}, - InitCPU: types.CPUMap{"0": 100, "1": 100}, - MemCap: 1000, - InitMemCap: 1000, - }, - }, - } - _, _, total, err = k.SelectCPUNodes(context.TODO(), scheduleInfos, 1, 50) - assert.EqualValues(t, 0, total) - assert.EqualError(t, err, "no node remains 1.00 pieces of cpu and 50 bytes of memory at the same time: not enough resource") - - // storage - scheduleInfos = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "n0", - StorageCap: -1000, - InitStorageCap: 1000, - }, - }, - } - _, total, err = k.SelectStorageNodes(context.TODO(), scheduleInfos, 500) - assert.EqualValues(t, 0, total) - assert.EqualError(t, err, "no node remains storage more than 500 bytes: cannot alloc a plan, not enough storage") - - // volume - scheduleInfos = []resourcetypes.ScheduleInfo{ - { - NodeMeta: types.NodeMeta{ - Name: "n0", - Volume: types.VolumeMap{"/tmp": -1234}, - InitVolume: types.VolumeMap{"/tmp": 100000}, - }, - }, - } - _, _, total, err = k.SelectVolumeNodes(context.TODO(), scheduleInfos, types.MustToVolumeBindings([]string{"AUTO:/data:rw:1"})) - assert.EqualValues(t, 0, total) - assert.EqualError(t, err, "no node remains volumes for requests [AUTO:/data:rw:1]: not enough resource") -} diff --git a/scheduler/complex/resource.go b/scheduler/complex/resource.go deleted file mode 100644 index 7527daa39..000000000 --- a/scheduler/complex/resource.go +++ /dev/null @@ -1,285 +0,0 @@ -package complexscheduler - -import ( - "container/heap" - "sort" - - "github.com/projecteru2/core/types" -) - -type resourceInfo struct { - id string - pieces int64 -} - -type resourceInfoHeap []resourceInfo - -// Len . -func (r resourceInfoHeap) Len() int { - return len(r) -} - -// Less . -func (r resourceInfoHeap) Less(i, j int) bool { - return r[i].pieces > r[j].pieces -} - -// Swap . -func (r resourceInfoHeap) Swap(i, j int) { - r[i], r[j] = r[j], r[i] -} - -// Push . -func (r *resourceInfoHeap) Push(x interface{}) { - *r = append(*r, x.(resourceInfo)) -} - -// Pop . -func (r *resourceInfoHeap) Pop() interface{} { - old := *r - n := len(old) - x := old[n-1] - *r = old[:n-1] - return x -} - -type host struct { - full []resourceInfo - fragment []resourceInfo - share int -} - -func newHost(resourceMap types.ResourceMap, share int) *host { - result := &host{ - share: share, - full: []resourceInfo{}, - fragment: []resourceInfo{}, - } - for id, pieces := range resourceMap { - // 整数核不应该切分 - if pieces >= int64(share) && pieces%int64(share) == 0 { - // 只给 share 份 - result.full = append(result.full, resourceInfo{id: id, pieces: pieces}) - } else if pieces > 0 { - result.fragment = append(result.fragment, resourceInfo{id: id, pieces: pieces}) - } - } - // 确保优先分配更碎片的核 - sort.Slice(result.fragment, func(i, j int) bool { return result.fragment[i].pieces < result.fragment[j].pieces }) - // 确保优先分配负重更大的整数核 - sort.Slice(result.full, func(i, j int) bool { return result.full[i].pieces < result.full[j].pieces }) - - return result -} - -func (h *host) getComplexResult(full int, fragment int64, maxShare int) []types.ResourceMap { - if maxShare == -1 { - maxShare = len(h.full) - full // 减枝,M == N 的情况下预留至少一个 full 量的核数 - } else { - maxShare -= len(h.fragment) - } - - // 计算默认情况下能部署多少个 - fragmentResultBase := h.getFragmentResult(fragment, h.fragment) - fullResultBase := h.getFullResult(full, h.full) - fragmentResultCount := len(fragmentResultBase) - fullResultCount := len(fullResultBase) - - baseLine := min(fragmentResultCount, fullResultCount) - fragmentResult := fragmentResultBase - fullResult := fullResultBase - for i := 1; i < maxShare+1; i++ { - fragmentResultBase = h.getFragmentResult(fragment, append(h.fragment, h.full[:i]...)) - fullResultBase = h.getFullResult(full, h.full[i:]) - fragmentResultCount = len(fragmentResultBase) - fullResultCount = len(fullResultBase) - - canDeployNum := min(fragmentResultCount, fullResultCount) - if canDeployNum > baseLine { - baseLine = canDeployNum - fragmentResult = fragmentResultBase - fullResult = fullResultBase - } - } - - result := []types.ResourceMap{} - for i := 0; i < baseLine; i++ { - r := types.ResourceMap{} - for id, pieces := range fullResult[i] { - r[id] += pieces - } - for id, pieces := range fragmentResult[i] { - r[id] = pieces - } - result = append(result, r) - } - - return result -} - -func (h *host) getFragmentResult(fragment int64, resources []resourceInfo) []types.ResourceMap { - resourceMaps := h.getFragmentsResult(resources, fragment) - result := make([]types.ResourceMap, len(resourceMaps)) - for i, resourceMap := range resourceMaps { - result[i] = resourceMap[0] - } - - // to pass tests due to new algorithm returns unsorted list - resourceIdx := map[string]int{} - for idx, resource := range resources { - resourceIdx[resource.id] = idx - } - sort.Slice(result, func(i, j int) bool { - return resourceIdx[result[i].GetResourceID()] < resourceIdx[result[j].GetResourceID()] - }) - return result -} - -func (h *host) getFragmentsResult(resources []resourceInfo, fragments ...int64) (resourceMap [][]types.ResourceMap) { - if len(fragments) == 0 { - return - } - - // shouldn't change resources slice - resourcesBak := make([]resourceInfo, len(resources)) - copy(resourcesBak, resources) - defer func() { - copy(resources, resourcesBak) - }() - - nFragments := len(fragments) - nResources := len(resources) - sort.Slice(fragments, func(i, j int) bool { return fragments[i] > fragments[j] }) // fragments is descendant - var totalRequired int64 - for _, fragment := range fragments { - totalRequired += fragment - } - - for i := 0; i < nResources; i++ { - sort.Slice(resources, func(i, j int) bool { return resources[i].pieces < resources[j].pieces }) - - count := resources[i].pieces / totalRequired - - // plan on the same resource - plan := []types.ResourceMap{} - for _, fragment := range fragments { - plan = append(plan, types.ResourceMap{resources[i].id: fragment}) - } - for j := int64(0); j < count; j++ { - resourceMap = append(resourceMap, plan) - } - resources[i].pieces -= count * totalRequired - - // plan on different resources - plan = []types.ResourceMap{} - refugees := []int64{} // refugees record the fragments not able to scheduled on resource[i] - for _, fragment := range fragments { - if resources[i].pieces >= fragment { - resources[i].pieces -= fragment - plan = append(plan, types.ResourceMap{resources[i].id: fragment}) - } else { - refugees = append(refugees, fragment) - } - } - - if len(refugees) == nFragments { - // resources[i] runs out of capacity, calculate resources[i+1] - continue - } - - // looking for resource(s) capable of taking in refugees - for j := i + 1; j < nResources; j++ { - fragments := refugees - refugees = []int64{} - for _, fragment := range fragments { - if resources[j].pieces >= fragment { - resources[j].pieces -= fragment - plan = append(plan, types.ResourceMap{resources[j].id: fragment}) - } else { - refugees = append(refugees, fragment) - } - } - if len(refugees) == 0 { - resourceMap = append(resourceMap, plan) - break - } - } - if len(refugees) > 0 { - // fail to complete this plan - break - } - } - - return resourceMap -} - -func (h *host) getFullResult(full int, resources []resourceInfo) []types.ResourceMap { - result := []types.ResourceMap{} - resourceHeap := &resourceInfoHeap{} - indexMap := map[string]int{} - for i, resource := range resources { - indexMap[resource.id] = i - resourceHeap.Push(resourceInfo{id: resource.id, pieces: resource.pieces}) - } - heap.Init(resourceHeap) - - for resourceHeap.Len() >= full { - plan := types.ResourceMap{} - resourcesToPush := []resourceInfo{} - - for i := 0; i < full; i++ { - resource := heap.Pop(resourceHeap).(resourceInfo) - plan[resource.id] = int64(h.share) - - resource.pieces -= int64(h.share) - if resource.pieces > 0 { - resourcesToPush = append(resourcesToPush, resource) - } - } - - result = append(result, plan) - for _, resource := range resourcesToPush { - heap.Push(resourceHeap, resource) - } - } - - // Try to ensure the effectiveness of the previous priority - sumOfIds := func(r types.ResourceMap) int { - sum := 0 - for id := range r { - sum += indexMap[id] - } - return sum - } - - sort.Slice(result, func(i, j int) bool { return sumOfIds(result[i]) < sumOfIds(result[j]) }) - - return result -} - -func (h *host) distributeOneRation(ration float64, maxShare int) []types.ResourceMap { - ration *= float64(h.share) - fullRequire := int64(ration) / int64(h.share) - fragmentRequire := int64(ration) % int64(h.share) - - if fullRequire == 0 { - if maxShare == -1 { - // 这个时候就把所有的资源都当成碎片 - maxShare = len(h.full) + len(h.fragment) - } - diff := maxShare - len(h.fragment) - h.fragment = append(h.fragment, h.full[:diff]...) - - return h.getFragmentResult(fragmentRequire, h.fragment) - } - - if fragmentRequire == 0 { - return h.getFullResult(int(fullRequire), h.full) - } - - return h.getComplexResult(int(fullRequire), fragmentRequire, maxShare) -} - -func (h *host) distributeMultipleRations(rations []int64) [][]types.ResourceMap { - return h.getFragmentsResult(h.fragment, rations...) -} diff --git a/scheduler/complex/utils.go b/scheduler/complex/utils.go deleted file mode 100644 index d5d7075e5..000000000 --- a/scheduler/complex/utils.go +++ /dev/null @@ -1,29 +0,0 @@ -package complexscheduler - -import ( - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/types" - "github.com/projecteru2/core/utils" -) - -func updateScheduleInfoCapacity(scheduleInfo *resourcetypes.ScheduleInfo, capacity int) int { - if scheduleInfo.Capacity == 0 { - scheduleInfo.Capacity = capacity - } else { - scheduleInfo.Capacity = utils.Min(capacity, scheduleInfo.Capacity) - } - return scheduleInfo.Capacity -} - -func onSameSource(plan []types.ResourceMap) bool { - sourceID := "" - for _, p := range plan { - if sourceID == "" { - sourceID = p.GetResourceID() - } - if sourceID != p.GetResourceID() { - return false - } - } - return true -} diff --git a/scheduler/complex/volume.go b/scheduler/complex/volume.go deleted file mode 100644 index fa6487584..000000000 --- a/scheduler/complex/volume.go +++ /dev/null @@ -1,79 +0,0 @@ -package complexscheduler - -import ( - "math" - - "github.com/projecteru2/core/types" -) - -func calculateVolumePlan(volumeMap types.VolumeMap, required []int64) (int, [][]types.VolumeMap) { - if len(required) == 0 { - return math.MaxInt64, nil - } - - share := int(math.MaxInt64) // all fragments - host := newHost(volumeMap, share) - plans := host.distributeMultipleRations(required) - return len(plans), plans -} - -func calculateMonopolyVolumePlan(initVolumeMap types.VolumeMap, volumeMap types.VolumeMap, required []int64) (cap int, plans [][]types.VolumeMap) { - cap, rawPlans := calculateVolumePlan(volumeMap, required) - if rawPlans == nil { - return cap, nil - } - - scheduled := map[string]bool{} - for _, plan := range rawPlans { - if !onSameSource(plan) { - continue - } - - volume := plan[0].GetResourceID() - if _, ok := scheduled[volume]; ok { - continue - } - - plans = append(plans, proportionPlan(plan, initVolumeMap[volume])) - scheduled[volume] = true - } - return len(plans), plans -} - -func proportionPlan(plan []types.VolumeMap, size int64) (newPlan []types.VolumeMap) { - var total int64 - for _, p := range plan { - total += p.GetRation() - } - for _, p := range plan { - newRation := int64(math.Floor(float64(p.GetRation()) / float64(total) * float64(size))) - newPlan = append(newPlan, types.VolumeMap{p.GetResourceID(): newRation}) - } - return -} - -func distinguishVolumeBindings(vbs types.VolumeBindings) (norm, mono, unlim types.VolumeBindings) { - for _, vb := range vbs { - switch { - case vb.RequireScheduleMonopoly(): - mono = append(mono, vb) - case vb.RequireScheduleUnlimitedQuota(): - unlim = append(unlim, vb) - case vb.RequireSchedule(): - norm = append(norm, vb) - } - } - return -} - -func distinguishAffinityVolumeBindings(vbs types.VolumeBindings, existing types.VolumePlan) (requireAff, remain types.VolumeBindings) { - for _, vb := range vbs { - _, _, found := existing.FindAffinityPlan(*vb) - if found { - requireAff = append(requireAff, vb) - } else { - remain = append(remain, vb) - } - } - return -} diff --git a/scheduler/complex/volume_test.go b/scheduler/complex/volume_test.go deleted file mode 100644 index f7c537abc..000000000 --- a/scheduler/complex/volume_test.go +++ /dev/null @@ -1,236 +0,0 @@ -package complexscheduler - -import ( - "context" - "reflect" - "testing" - - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/types" - - "github.com/stretchr/testify/assert" -) - -func TestVolumeRealloc(t *testing.T) { - po, err := New(types.Config{}) - assert.Nil(t, err) - - // affinity: no change - scheduleInfo := func() resourcetypes.ScheduleInfo { - return resourcetypes.ScheduleInfo{ - NodeMeta: types.NodeMeta{ - Name: "n1", - InitVolume: types.VolumeMap{ - "/tmp1": 100, - "/tmp2": 100, - "/tmp3": 100, - "/tmp4": 100, - "/tmp5": 100, - }, - Volume: types.VolumeMap{ - "/tmp1": 100, - "/tmp2": 100, - "/tmp3": 100, - "/tmp4": 100, - "/tmp5": 100, - }, - }, - } - } - existing := types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rmw:1": { - "/tmp3": 100, - }, - "AUTO:/data2:r:2": { - "/tmp5": 2, - }, - "AUTO:/data3:rw": { - "/tmp3": 0, - }, - }) - req := types.MustToVolumeBindings([]string{"AUTO:/data1:rmw:1", "AUTO:/data2:r:2", "AUTO:/data3:rw"}) - si, volumePlans, total, err := po.ReselectVolumeNodes(context.TODO(), scheduleInfo(), existing, req) - assert.Nil(t, err) - assert.EqualValues(t, 1, total) - assert.EqualValues(t, 1, si.Capacity) - assert.True(t, reflect.DeepEqual(volumePlans[scheduleInfo().Name][0], existing)) - - // affinity: complex no change - existing = types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rmw:1": { - "/tmp3": 50, - }, - "AUTO:/data2:r:2": { - "/tmp5": 2, - }, - "AUTO:/data3:rw": { - "/tmp3": 0, - }, - "AUTO:/data4:rmw:1": { - "/tmp3": 50, - }, - "AUTO:/data5:w:20": { - "/tmp5": 20, - }, - }) - req = types.MustToVolumeBindings([]string{ - "AUTO:/data1:rmw:1", - "AUTO:/data2:r:2", - "AUTO:/data3:rw", - "AUTO:/data4:rmw:1", - "AUTO:/data5:w:20", - }) - si, volumePlans, total, err = po.ReselectVolumeNodes(context.TODO(), scheduleInfo(), existing, req) - assert.Nil(t, err) - assert.EqualValues(t, 1, total) - assert.EqualValues(t, 1, si.Capacity) - assert.True(t, reflect.DeepEqual(volumePlans[scheduleInfo().Name][0], existing)) - - // affinity: increase - existing = types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rmw:1": { - "/tmp3": 100, - }, - "AUTO:/data2:r:2": { - "/tmp5": 2, - }, - }) - req = types.MustToVolumeBindings([]string{"AUTO:/data1:rmw:2", "AUTO:/data2:r:30"}) - si, volumePlans, total, err = po.ReselectVolumeNodes(context.TODO(), scheduleInfo(), existing, req) - assert.Nil(t, err) - assert.EqualValues(t, 1, total) - assert.EqualValues(t, 1, si.Capacity) - assert.True(t, reflect.DeepEqual(volumePlans[scheduleInfo().Name][0], types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rmw:2": { - "/tmp3": 100, - }, - "AUTO:/data2:r:30": { - "/tmp5": 30, - }, - }))) - - // affinity: decrease - existing = types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rmw:2": { - "/tmp3": 50, - }, - "AUTO:/data2:r:54": { - "/tmp5": 54, - }, - "AUTO:/data3:rmw:2": { - "/tmp3": 50, - }, - "AUTO:/data4:r:34": { - "/tmp5": 34, - }, - }) - req = types.MustToVolumeBindings([]string{ - "AUTO:/data1:rmw:1", - "AUTO:/data2:r:30", - "AUTO:/data3:rmw:3", - "AUTO:/data4:r:31", - }) - si, volumePlans, total, err = po.ReselectVolumeNodes(context.TODO(), scheduleInfo(), existing, req) - assert.Nil(t, err) - assert.EqualValues(t, 1, total) - assert.EqualValues(t, 1, si.Capacity) - assert.True(t, reflect.DeepEqual(volumePlans[scheduleInfo().Name][0], types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rmw:1": { - "/tmp3": 25, - }, - "AUTO:/data2:r:30": { - "/tmp5": 30, - }, - "AUTO:/data3:rmw:3": { - "/tmp3": 75, - }, - "AUTO:/data4:r:31": { - "/tmp5": 31, - }, - }))) - - // add some new volumes - existing = types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data2:r": { - "/tmp5": 0, - }, - }) - req = types.MustToVolumeBindings([]string{ - "AUTO:/data1:rmw:1", - "AUTO:/data2:r", - "AUTO:/data3:rmw:3", - }) - si, volumePlans, total, err = po.ReselectVolumeNodes(context.TODO(), scheduleInfo(), existing, req) - assert.Nil(t, err) - assert.EqualValues(t, 5, total) - assert.EqualValues(t, 5, si.Capacity) - assert.True(t, reflect.DeepEqual(volumePlans[scheduleInfo().Name][0][types.MustToVolumeBinding("AUTO:/data2:r")], types.VolumeMap{"/tmp5": 0})) - - // del some volumes - existing = types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rmw:2": { - "/tmp3": 50, - }, - "AUTO:/data2:r:54": { - "/tmp5": 54, - }, - "AUTO:/data3:rmw:2": { - "/tmp3": 50, - }, - "AUTO:/data4:r:34": { - "/tmp5": 34, - }, - }) - req = types.MustToVolumeBindings([]string{ - "AUTO:/data1:rmw:1", - "AUTO:/data2:r:30", - }) - si, volumePlans, total, err = po.ReselectVolumeNodes(context.TODO(), scheduleInfo(), existing, req) - assert.Nil(t, err) - assert.EqualValues(t, 1, total) - assert.EqualValues(t, 1, si.Capacity) - assert.True(t, reflect.DeepEqual(volumePlans[scheduleInfo().Name][0], types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rmw:1": { - "/tmp3": 100, - }, - "AUTO:/data2:r:30": { - "/tmp5": 30, - }, - }))) - - // expand mono error - existing = types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rmw:2": { - "/tmp3": 100, - }, - }) - req = types.MustToVolumeBindings([]string{ - "AUTO:/data1:rmw:2", - "AUTO:/data2:rmw:99", - }) - si, volumePlans, total, err = po.ReselectVolumeNodes(context.TODO(), scheduleInfo(), existing, req) - assert.Error(t, err, "no space to expand mono volumes") - - // expand norm error - existing = types.MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:rw:2": { - "/tmp3": 2, - }, - "AUTO:/data2:rw:2": { - "/tmp3": 2, - }, - }) - req = types.MustToVolumeBindings([]string{ - "AUTO:/data1:rw:3", - "AUTO:/data2:rw:98", - }) - si, volumePlans, total, err = po.ReselectVolumeNodes(context.TODO(), scheduleInfo(), existing, req) - assert.Error(t, err, "no space to expand") - - // reschedule err - req = types.MustToVolumeBindings([]string{ - "AUTO:/data1:rw:200", - }) - si, volumePlans, total, err = po.ReselectVolumeNodes(context.TODO(), scheduleInfo(), nil, req) - assert.Error(t, err, "failed to reschedule") -} diff --git a/scheduler/mocks/Scheduler.go b/scheduler/mocks/Scheduler.go deleted file mode 100644 index 83b4f41ec..000000000 --- a/scheduler/mocks/Scheduler.go +++ /dev/null @@ -1,252 +0,0 @@ -// Code generated by mockery v0.0.0-dev. DO NOT EDIT. - -package mocks - -import ( - context "context" - - resourcestypes "github.com/projecteru2/core/resources/types" - mock "github.com/stretchr/testify/mock" - - types "github.com/projecteru2/core/types" -) - -// Scheduler is an autogenerated mock type for the Scheduler type -type Scheduler struct { - mock.Mock -} - -// MaxIdleNode provides a mock function with given fields: nodes -func (_m *Scheduler) MaxIdleNode(nodes []*types.Node) (*types.Node, error) { - ret := _m.Called(nodes) - - var r0 *types.Node - if rf, ok := ret.Get(0).(func([]*types.Node) *types.Node); ok { - r0 = rf(nodes) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*types.Node) - } - } - - var r1 error - if rf, ok := ret.Get(1).(func([]*types.Node) error); ok { - r1 = rf(nodes) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// ReselectCPUNodes provides a mock function with given fields: ctx, scheduleInfo, CPU, quota, memory -func (_m *Scheduler) ReselectCPUNodes(ctx context.Context, scheduleInfo resourcestypes.ScheduleInfo, CPU types.ResourceMap, quota float64, memory int64) (resourcestypes.ScheduleInfo, map[string][]types.ResourceMap, int, error) { - ret := _m.Called(ctx, scheduleInfo, CPU, quota, memory) - - var r0 resourcestypes.ScheduleInfo - if rf, ok := ret.Get(0).(func(context.Context, resourcestypes.ScheduleInfo, types.ResourceMap, float64, int64) resourcestypes.ScheduleInfo); ok { - r0 = rf(ctx, scheduleInfo, CPU, quota, memory) - } else { - r0 = ret.Get(0).(resourcestypes.ScheduleInfo) - } - - var r1 map[string][]types.ResourceMap - if rf, ok := ret.Get(1).(func(context.Context, resourcestypes.ScheduleInfo, types.ResourceMap, float64, int64) map[string][]types.ResourceMap); ok { - r1 = rf(ctx, scheduleInfo, CPU, quota, memory) - } else { - if ret.Get(1) != nil { - r1 = ret.Get(1).(map[string][]types.ResourceMap) - } - } - - var r2 int - if rf, ok := ret.Get(2).(func(context.Context, resourcestypes.ScheduleInfo, types.ResourceMap, float64, int64) int); ok { - r2 = rf(ctx, scheduleInfo, CPU, quota, memory) - } else { - r2 = ret.Get(2).(int) - } - - var r3 error - if rf, ok := ret.Get(3).(func(context.Context, resourcestypes.ScheduleInfo, types.ResourceMap, float64, int64) error); ok { - r3 = rf(ctx, scheduleInfo, CPU, quota, memory) - } else { - r3 = ret.Error(3) - } - - return r0, r1, r2, r3 -} - -// ReselectVolumeNodes provides a mock function with given fields: ctx, scheduleInfo, old, request -func (_m *Scheduler) ReselectVolumeNodes(ctx context.Context, scheduleInfo resourcestypes.ScheduleInfo, old types.VolumePlan, request types.VolumeBindings) (resourcestypes.ScheduleInfo, map[string][]types.VolumePlan, int, error) { - ret := _m.Called(ctx, scheduleInfo, old, request) - - var r0 resourcestypes.ScheduleInfo - if rf, ok := ret.Get(0).(func(context.Context, resourcestypes.ScheduleInfo, types.VolumePlan, types.VolumeBindings) resourcestypes.ScheduleInfo); ok { - r0 = rf(ctx, scheduleInfo, old, request) - } else { - r0 = ret.Get(0).(resourcestypes.ScheduleInfo) - } - - var r1 map[string][]types.VolumePlan - if rf, ok := ret.Get(1).(func(context.Context, resourcestypes.ScheduleInfo, types.VolumePlan, types.VolumeBindings) map[string][]types.VolumePlan); ok { - r1 = rf(ctx, scheduleInfo, old, request) - } else { - if ret.Get(1) != nil { - r1 = ret.Get(1).(map[string][]types.VolumePlan) - } - } - - var r2 int - if rf, ok := ret.Get(2).(func(context.Context, resourcestypes.ScheduleInfo, types.VolumePlan, types.VolumeBindings) int); ok { - r2 = rf(ctx, scheduleInfo, old, request) - } else { - r2 = ret.Get(2).(int) - } - - var r3 error - if rf, ok := ret.Get(3).(func(context.Context, resourcestypes.ScheduleInfo, types.VolumePlan, types.VolumeBindings) error); ok { - r3 = rf(ctx, scheduleInfo, old, request) - } else { - r3 = ret.Error(3) - } - - return r0, r1, r2, r3 -} - -// SelectCPUNodes provides a mock function with given fields: ctx, scheduleInfos, quota, memory -func (_m *Scheduler) SelectCPUNodes(ctx context.Context, scheduleInfos []resourcestypes.ScheduleInfo, quota float64, memory int64) ([]resourcestypes.ScheduleInfo, map[string][]types.ResourceMap, int, error) { - ret := _m.Called(ctx, scheduleInfos, quota, memory) - - var r0 []resourcestypes.ScheduleInfo - if rf, ok := ret.Get(0).(func(context.Context, []resourcestypes.ScheduleInfo, float64, int64) []resourcestypes.ScheduleInfo); ok { - r0 = rf(ctx, scheduleInfos, quota, memory) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]resourcestypes.ScheduleInfo) - } - } - - var r1 map[string][]types.ResourceMap - if rf, ok := ret.Get(1).(func(context.Context, []resourcestypes.ScheduleInfo, float64, int64) map[string][]types.ResourceMap); ok { - r1 = rf(ctx, scheduleInfos, quota, memory) - } else { - if ret.Get(1) != nil { - r1 = ret.Get(1).(map[string][]types.ResourceMap) - } - } - - var r2 int - if rf, ok := ret.Get(2).(func(context.Context, []resourcestypes.ScheduleInfo, float64, int64) int); ok { - r2 = rf(ctx, scheduleInfos, quota, memory) - } else { - r2 = ret.Get(2).(int) - } - - var r3 error - if rf, ok := ret.Get(3).(func(context.Context, []resourcestypes.ScheduleInfo, float64, int64) error); ok { - r3 = rf(ctx, scheduleInfos, quota, memory) - } else { - r3 = ret.Error(3) - } - - return r0, r1, r2, r3 -} - -// SelectMemoryNodes provides a mock function with given fields: ctx, scheduleInfos, quota, memory -func (_m *Scheduler) SelectMemoryNodes(ctx context.Context, scheduleInfos []resourcestypes.ScheduleInfo, quota float64, memory int64) ([]resourcestypes.ScheduleInfo, int, error) { - ret := _m.Called(ctx, scheduleInfos, quota, memory) - - var r0 []resourcestypes.ScheduleInfo - if rf, ok := ret.Get(0).(func(context.Context, []resourcestypes.ScheduleInfo, float64, int64) []resourcestypes.ScheduleInfo); ok { - r0 = rf(ctx, scheduleInfos, quota, memory) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]resourcestypes.ScheduleInfo) - } - } - - var r1 int - if rf, ok := ret.Get(1).(func(context.Context, []resourcestypes.ScheduleInfo, float64, int64) int); ok { - r1 = rf(ctx, scheduleInfos, quota, memory) - } else { - r1 = ret.Get(1).(int) - } - - var r2 error - if rf, ok := ret.Get(2).(func(context.Context, []resourcestypes.ScheduleInfo, float64, int64) error); ok { - r2 = rf(ctx, scheduleInfos, quota, memory) - } else { - r2 = ret.Error(2) - } - - return r0, r1, r2 -} - -// SelectStorageNodes provides a mock function with given fields: ctx, scheduleInfos, storage -func (_m *Scheduler) SelectStorageNodes(ctx context.Context, scheduleInfos []resourcestypes.ScheduleInfo, storage int64) ([]resourcestypes.ScheduleInfo, int, error) { - ret := _m.Called(ctx, scheduleInfos, storage) - - var r0 []resourcestypes.ScheduleInfo - if rf, ok := ret.Get(0).(func(context.Context, []resourcestypes.ScheduleInfo, int64) []resourcestypes.ScheduleInfo); ok { - r0 = rf(ctx, scheduleInfos, storage) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]resourcestypes.ScheduleInfo) - } - } - - var r1 int - if rf, ok := ret.Get(1).(func(context.Context, []resourcestypes.ScheduleInfo, int64) int); ok { - r1 = rf(ctx, scheduleInfos, storage) - } else { - r1 = ret.Get(1).(int) - } - - var r2 error - if rf, ok := ret.Get(2).(func(context.Context, []resourcestypes.ScheduleInfo, int64) error); ok { - r2 = rf(ctx, scheduleInfos, storage) - } else { - r2 = ret.Error(2) - } - - return r0, r1, r2 -} - -// SelectVolumeNodes provides a mock function with given fields: ctx, scheduleInfo, vbs -func (_m *Scheduler) SelectVolumeNodes(ctx context.Context, scheduleInfo []resourcestypes.ScheduleInfo, vbs types.VolumeBindings) ([]resourcestypes.ScheduleInfo, map[string][]types.VolumePlan, int, error) { - ret := _m.Called(ctx, scheduleInfo, vbs) - - var r0 []resourcestypes.ScheduleInfo - if rf, ok := ret.Get(0).(func(context.Context, []resourcestypes.ScheduleInfo, types.VolumeBindings) []resourcestypes.ScheduleInfo); ok { - r0 = rf(ctx, scheduleInfo, vbs) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]resourcestypes.ScheduleInfo) - } - } - - var r1 map[string][]types.VolumePlan - if rf, ok := ret.Get(1).(func(context.Context, []resourcestypes.ScheduleInfo, types.VolumeBindings) map[string][]types.VolumePlan); ok { - r1 = rf(ctx, scheduleInfo, vbs) - } else { - if ret.Get(1) != nil { - r1 = ret.Get(1).(map[string][]types.VolumePlan) - } - } - - var r2 int - if rf, ok := ret.Get(2).(func(context.Context, []resourcestypes.ScheduleInfo, types.VolumeBindings) int); ok { - r2 = rf(ctx, scheduleInfo, vbs) - } else { - r2 = ret.Get(2).(int) - } - - var r3 error - if rf, ok := ret.Get(3).(func(context.Context, []resourcestypes.ScheduleInfo, types.VolumeBindings) error); ok { - r3 = rf(ctx, scheduleInfo, vbs) - } else { - r3 = ret.Error(3) - } - - return r0, r1, r2, r3 -} diff --git a/scheduler/scheduler.go b/scheduler/scheduler.go deleted file mode 100644 index bb5d7c81d..000000000 --- a/scheduler/scheduler.go +++ /dev/null @@ -1,47 +0,0 @@ -package scheduler - -import ( - "context" - - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/types" - - "github.com/pkg/errors" -) - -var ( - scheduler Scheduler -) - -// Scheduler is a scheduler is used to determine which nodes are we gonna use. -// `types.CPUMap` represents the CPU label and remaining quota. -// `nodes` represents node name and the corresponding CPUMap. -type Scheduler interface { - // select one node from nodes, returns nodename - // typically used to build image - MaxIdleNode(nodes []*types.Node) (*types.Node, error) - SelectStorageNodes(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo, storage int64) ([]resourcetypes.ScheduleInfo, int, error) - SelectMemoryNodes(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo, quota float64, memory int64) ([]resourcetypes.ScheduleInfo, int, error) - // select nodes from nodes, return a list of nodenames and the corresponding cpumap, and also the changed nodes with remaining cpumap - // quota and number must be given, typically used to determine where to deploy - SelectCPUNodes(ctx context.Context, scheduleInfos []resourcetypes.ScheduleInfo, quota float64, memory int64) ([]resourcetypes.ScheduleInfo, map[string][]types.CPUMap, int, error) - // ReselectCPUNodes is used for realloc only - ReselectCPUNodes(ctx context.Context, scheduleInfo resourcetypes.ScheduleInfo, CPU types.CPUMap, quota float64, memory int64) (resourcetypes.ScheduleInfo, map[string][]types.CPUMap, int, error) - // select nodes from nodes, return a list a nodenames and the corresponding volumemap - SelectVolumeNodes(ctx context.Context, scheduleInfo []resourcetypes.ScheduleInfo, vbs types.VolumeBindings) ([]resourcetypes.ScheduleInfo, map[string][]types.VolumePlan, int, error) - // ReselectVolumeNodes is used for realloc only - ReselectVolumeNodes(ctx context.Context, scheduleInfo resourcetypes.ScheduleInfo, old types.VolumePlan, request types.VolumeBindings) (resourcetypes.ScheduleInfo, map[string][]types.VolumePlan, int, error) -} - -// InitSchedulerV1 . -func InitSchedulerV1(s Scheduler) { - scheduler = s -} - -// GetSchedulerV1 . -func GetSchedulerV1() (Scheduler, error) { - if scheduler == nil { - return nil, errors.WithStack(errors.Errorf("potassium not initiated")) - } - return scheduler, nil -} diff --git a/scripts/meta_transfer_as_rename2workload.py b/scripts/meta_transfer_as_rename2workload.py index 89c26f4b3..1d7f1d104 100644 --- a/scripts/meta_transfer_as_rename2workload.py +++ b/scripts/meta_transfer_as_rename2workload.py @@ -4,14 +4,15 @@ import functools import json import os -import sys import etcd3 +import sys check_conflict = False dry_run = False override = False + def remove_prefix(s, prefix): return s[len(prefix):].lstrip('/') if s.startswith(prefix) else s @@ -267,6 +268,7 @@ def delete(*keys): pass del_keys = set() + def get(new_field, orig_field, transit_field, default=None): value = None @@ -337,6 +339,7 @@ def trans(self, podname=None): Workload(self, node_transfer, podname, nodes).trans() + def getargs(): ap = argparse.ArgumentParser() ap.add_argument('-o', '--orig', dest='orig_root_prefix', help='original prefix', default='/eru') @@ -345,13 +348,17 @@ def getargs(): ap.add_argument('--etcd-host', default='127.0.0.1') ap.add_argument('--etcd-port', type=int, default=2379) ap.add_argument('--dry-run', dest='dry_run', action='store_true', help='dry run, will not actually migrate') - ap.add_argument('--check-conflict', dest='check_conflict', action='store_true', help='check conflict, checks if destination already has the key') - ap.add_argument('--override', dest='override', action='store_true', help='if set, the value will be migrated anyway no matter if it already exists') + ap.add_argument('--check-conflict', dest='check_conflict', action='store_true', + help='check conflict, checks if destination already has the key') + ap.add_argument('--override', dest='override', action='store_true', + help='if set, the value will be migrated anyway no matter if it already exists') return ap.parse_args() + def connect_etcd(host, port): return etcd3.client(host=host, port=port) + def main(): args = getargs() @@ -367,5 +374,6 @@ def main(): trans.trans(args.podname) return 0 + if __name__ == '__main__': sys.exit(main()) diff --git a/scripts/meta_transfer_resource_plugin.py b/scripts/meta_transfer_resource_plugin.py new file mode 100644 index 000000000..d468ecec0 --- /dev/null +++ b/scripts/meta_transfer_resource_plugin.py @@ -0,0 +1,369 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- + +import argparse +import functools +import json +import os + +import etcd3 + +dry_run = False +record_prefix = "upgrade_" +origin_data_record_path = 'origin_data_record.data' +transferred_node_record_path = 'transferred_node_record.data' +transferred_workload_record_path = 'transferred_workload_record.data' + +origin_data_recorder = None +transferred_node_recorder = None +transferred_workload_recorder = None + +transferred_workloads = set() +transferred_nodes = set() + + +def init_recorders(): + global origin_data_recorder, origin_data_record_path + global transferred_node_recorder, transferred_node_record_path, transferred_nodes + global transferred_workload_recorder, transferred_workload_record_path, transferred_workloads + + origin_data_record_path = record_prefix + origin_data_record_path + transferred_node_record_path = record_prefix + transferred_node_record_path + transferred_workload_record_path = record_prefix + transferred_workload_record_path + + if os.path.exists(transferred_node_record_path): + with open(transferred_node_record_path, 'r') as f: + transferred_nodes = set(f.read().strip('\n').splitlines()) + + if os.path.exists(transferred_workload_record_path): + with open(transferred_workload_record_path, 'r') as f: + transferred_workloads = set(f.read().strip('\n').splitlines()) + + origin_data_recorder = open(origin_data_record_path, 'a') + transferred_node_recorder = open(transferred_node_record_path, 'a') + transferred_workload_recorder = open(transferred_workload_record_path, 'a') + + +def close_recorders(): + transferred_node_recorder.close() + transferred_workload_recorder.close() + origin_data_recorder.close() + + +def add_record(recorder, record): + recorder.write('%s\n' % record) + + +def remove_prefix(s, prefix): + return s[len(prefix):].lstrip('/') if s.startswith(prefix) else s + + +def dict_sub(d1, d2): + if d1 is None: + return None + if d2 is None: + return d1 + get = lambda d, k: d[k] if k in d else 0 + return {k: d1[k] - get(d2, k) for k in d1} + + +class ETCD: + def __init__(self, client, prefix): + """Create an instance of ETCD.""" + self.etcd = client + self.prefix = prefix + + def get(self, key): + if not key.startswith(self.prefix): + key = self.prefix + key + res = self.etcd.get(key)[0] + if res is None: + return None + return res.decode('utf-8') + + def put(self, key, value): + if not key.startswith(self.prefix): + key = self.prefix + key + if dry_run: + print('put {}\n{}'.format(key, value)) + return + + origin_value = self.get(key) + if origin_value: + add_record(origin_data_recorder, key) + add_record(origin_data_recorder, origin_value) + + self.etcd.put(key, value) + + def range_prefix(self, obj_prefix, fn): + prefix = self.prefix + obj_prefix + range_start = prefix + range_end = etcd3.utils.increment_last_byte( + etcd3.utils.to_bytes(range_start) + ) + + while True: + range_request = etcd3.etcdrpc.RangeRequest() + range_request.key = etcd3.utils.to_bytes(range_start) + range_request.keys_only = False + range_request.range_end = etcd3.utils.to_bytes(range_end) + range_request.sort_order = etcd3.etcdrpc.RangeRequest.ASCEND + range_request.sort_target = etcd3.etcdrpc.RangeRequest.KEY + range_request.serializable = True + range_request.limit = 1000 + + range_response = self.etcd.kvstub.Range( + range_request, + self.etcd.timeout, + credentials=self.etcd.call_credentials, + metadata=self.etcd.metadata, + ) + + for kv in range_response.kvs: + orig_key = kv.key.decode('utf-8') + objname = remove_prefix(orig_key, prefix) + fn(objname, kv.value.decode('utf-8')) + + if not range_response.more: + break + + range_start = etcd3.utils.increment_last_byte(kv.key) + + +etcd: ETCD = None + + +class Node: + def __init__(self, name, pod_name, meta): + """Initializes a node transfer.""" + self.name = name + self.pod_name = pod_name + self.meta = json.loads(meta) + + def upgrade(self): + cpumem_meta = self._gen_cpumem_meta() + volume_meta = self._gen_volume_meta() + cpumem_key = '/resource/cpumem/%s' % self.name + volume_key = '/resource/volume/%s' % self.name + etcd.put(cpumem_key, cpumem_meta) + etcd.put(volume_key, volume_meta) + + def downgrade(self): + self._load_resources_meta() + keys = ['/node/%s' % self.name, '/node/%s:pod/%s' % (self.pod_name, self.name)] + for key in keys: + etcd.put(key, json.dumps(self.meta)) + + def _load_cpumem_meta(self, meta): + cpumem_meta = json.loads(meta) + self.meta['init_cpu'] = cpumem_meta['capacity']['cpu_map'] + self.meta['cpu'] = dict_sub(cpumem_meta['capacity']['cpu_map'], cpumem_meta['usage']['cpu_map']) + self.meta['init_memcap'] = cpumem_meta['capacity']['memory'] + self.meta['memcap'] = cpumem_meta['capacity']['memory'] - cpumem_meta['usage']['memory'] + self.meta['cpuused'] = cpumem_meta['usage']['cpu'] + self.meta['numa'] = cpumem_meta['capacity']['numa'] + self.meta['init_numa_memory'] = cpumem_meta['capacity']['numa_memory'] + self.meta['numa_memory'] = dict_sub(cpumem_meta['capacity']['numa_memory'], cpumem_meta['usage']['numa_memory']) + + def _load_resources_meta(self): + # load cpumem resources + cpumem_key = '/resource/cpumem/%s' % self.name + cpumem_meta = etcd.get(cpumem_key) + if not cpumem_meta: + print("%s not found" % cpumem_key) + self._load_cpumem_meta(cpumem_meta) + + # load volume resources + volume_key = '/resource/volume/%s' % self.name + volume_meta = etcd.get(volume_key) + if not volume_meta: + print("%s not found" % volume_key) + self._load_volume_meta(volume_meta) + + def _load_volume_meta(self, meta): + volume_meta = json.loads(meta) + self.meta['init_volume'] = volume_meta['capacity']['volumes'] + self.meta['volume'] = volume_meta['usage']['volumes'] + self.meta['init_storage_cap'] = volume_meta['capacity']['storage'] + self.meta['storage_cap'] = volume_meta['usage']['storage'] + self.meta['volumeused'] = sum(volume_meta['usage']['volumes'].values()) + + def _gen_cpumem_meta(self): + cpumem_meta = {"capacity": {}, "usage": {}} + cpumem_meta['capacity']['cpu_map'] = self.meta['init_cpu'] + cpumem_meta['usage']['cpu_map'] = dict_sub(self.meta['init_cpu'], self.meta['cpu']) + cpumem_meta['capacity']['memory'] = self.meta['init_memcap'] + cpumem_meta['usage']['memory'] = self.meta['init_memcap'] - self.meta['memcap'] + cpumem_meta['capacity']['cpu'] = len(self.meta['init_cpu']) + cpumem_meta['usage']['cpu'] = self.meta['cpuused'] + cpumem_meta['capacity']['numa'] = self.meta['numa'] + cpumem_meta['capacity']['numa_memory'] = self.meta['init_numa_memory'] + cpumem_meta['usage']['numa_memory'] = dict_sub(self.meta['init_numa_memory'], self.meta['numa_memory']) + return json.dumps(cpumem_meta) + + def _gen_volume_meta(self): + volume_meta = {"capacity": {}, "usage": {}} + volume_meta['capacity']['volumes'] = self.meta['init_volume'] + volume_meta['usage']['volumes'] = dict_sub(self.meta['init_volume'], self.meta['volume']) + volume_meta['capacity']['storage'] = self.meta['init_storage_cap'] + volume_meta['usage']['storage'] = self.meta['init_storage_cap'] - self.meta['storage_cap'] + return json.dumps(volume_meta) + + +class Workload: + def __init__(self, workload_id, app_name, entry_name, node_name, meta): + """Initializes a workload transfer.""" + self.workload_id = workload_id + self.app_name = app_name + self.entry_name = entry_name + self.node_name = node_name + self.meta = json.loads(meta) + self.keys = ['/workloads/%s' % self.workload_id, + '/deploy/%s/%s/%s/%s' % (self.app_name, self.entry_name, self.node_name, self.workload_id), + '/node/%s:workloads/%s' % (self.node_name, self.workload_id)] + + def save(self): + for key in self.keys: + etcd.put(key, json.dumps(self.meta)) + + def upgrade(self): + if self.workload_id in transferred_workloads: + return + self._gen_resource_meta() + self.save() + + def downgrade(self): + if self.workload_id in transferred_workloads: + return + self._load_resource_meta() + self.save() + + def _gen_resource_meta(self): + self.meta['resource_args'] = {} + self.meta['resource_args']['cpumem'] = { + 'cpu_request': self.meta['cpu_quota_request'], + 'cpu_limit': self.meta['cpu_quota_limit'], + 'cpu_map': self.meta['cpu'], + 'memory_request': self.meta['memory_request'], + 'memory_limit': self.meta['memory_limit'], + "numa_node": self.meta['numa_node'], + } + self.meta['resource_args']['volume'] = { + 'volumes_request': self.meta['volume_request'], + 'volumes_limit': self.meta['volume_limit'], + 'volume_plan_request': self.meta['volume_plan_request'], + 'volume_plan_limit': self.meta['volume_plan_limit'], + 'storage_request': self.meta['storage_request'], + 'storage_limit': self.meta['storage_limit'], + } + self.meta['engine_args'] = { + 'cpu': self.meta['cpu_quota_limit'], + 'memory': self.meta['memory_limit'], + 'numa_node': self.meta['numa_node'], + 'cpu_map': self.meta['cpu'], + 'storage': self.meta['storage_limit'], + 'volume': [], + } + for binding in self.meta['volume_limit']: + if not binding.startswith('AUTO'): + self.meta['engine_args']['volume'].append(binding) + + for binding in self.meta['volume_plan_limit']: + groups = binding.split(':') + if len(groups) < 3: + print("volume plan limit of %s is invalid: %s" % (self.workload_id, binding)) + + dst = groups[1] + flags = groups[2] + device = list(self.meta['volume_plan_limit'][binding].keys())[0] + size = self.meta['volume_plan_limit'][binding][device] + flags = flags.replace('m', '') + if 'o' in flags: + flags = flags.replace('o', '').replace('r', 'ro').replace('w', 'wo') + + self.meta['engine_args']['volume'].append('%s:%s:%s:%s' % (device, dst, flags, size)) + + def _load_resource_meta(self): + self.meta['cpu_quota_request'] = self.meta['resource_args']['cpumem']['cpu_request'] + self.meta['cpu_quota_limit'] = self.meta['resource_args']['cpumem']['cpu_limit'] + self.meta['cpu'] = self.meta['resource_args']['cpumem']['cpu_map'] + self.meta['memory_request'] = self.meta['resource_args']['cpumem']['memory_request'] + self.meta['memory_limit'] = self.meta['resource_args']['cpumem']['memory_limit'] + self.meta['numa_node'] = self.meta['resource_args']['cpumem']['numa_node'] + self.meta['volume_request'] = self.meta['resource_args']['volume']['volumes_request'] + self.meta['volume_limit'] = self.meta['resource_args']['volume']['volumes_limit'] + self.meta['volume_plan_request'] = self.meta['resource_args']['volume']['volume_plan_request'] + self.meta['volume_plan_limit'] = self.meta['resource_args']['volume']['volume_plan_limit'] + self.meta['storage_request'] = self.meta['resource_args']['volume']['storage_request'] + self.meta['storage_limit'] = self.meta['resource_args']['volume']['storage_limit'] + + +def connect_etcd(host, port): + return etcd3.client(host=host, port=port) + + +def transfer_node(key, value, upgrade=True): + if ':pod' not in key: + return + node_name = key.split('/')[-1] + pod_name = key.split(':')[0].strip('/') + if node_name in transferred_nodes: + return + + print('transferring node %s' % node_name) + node = Node(node_name, pod_name, value) + if upgrade: + node.upgrade() + else: + node.downgrade() + add_record(transferred_node_recorder, node_name) + + +def transfer_workload(key, value, upgrade=True): + app_name, entry_name, node_name, workload_id = key.strip('/').split('/') + if workload_id in transferred_workloads: + return + + print('transferring workload %s' % workload_id) + workload = Workload(workload_id, app_name, entry_name, node_name, value) + if upgrade: + workload.upgrade() + else: + workload.downgrade() + add_record(transferred_workload_recorder, workload_id) + + +def transfer(upgrade=True): + etcd.range_prefix('/node', functools.partial(transfer_node, upgrade=upgrade)) + etcd.range_prefix('/deploy', functools.partial(transfer_workload, upgrade=upgrade)) + + +def get_args(): + ap = argparse.ArgumentParser() + ap.add_argument('--upgrade', action='store_true', help='upgrade to new eru-core') + ap.add_argument('--downgrade', action='store_true', help='downgrade to old eru-core') + ap.add_argument('--etcd-prefix', help='etcd prefix', default='/eru') + ap.add_argument('--etcd-host', default='127.0.0.1') + ap.add_argument('--etcd-port', type=int, default=2379) + ap.add_argument('--dry-run', dest='dry_run', action='store_true', help='dry run, will not actually migrate') + return ap.parse_args() + + +def main(): + args = get_args() + if not args.upgrade and not args.downgrade: + print('please specify --upgrade or --downgrade') + + global etcd, dry_run, record_prefix + etcd = ETCD(connect_etcd(args.etcd_host, args.etcd_port), args.etcd_prefix) + dry_run = args.dry_run + upgrade = args.upgrade + if not upgrade: + record_prefix = 'downgrade' + + init_recorders() + transfer(upgrade) + + +if __name__ == '__main__': + main() diff --git a/store/etcdv3/deploy.go b/store/etcdv3/deploy.go index ee34b12f0..579726612 100644 --- a/store/etcdv3/deploy.go +++ b/store/etcdv3/deploy.go @@ -5,30 +5,44 @@ import ( "path/filepath" "strings" - "github.com/projecteru2/core/log" - "github.com/projecteru2/core/strategy" - clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/projecteru2/core/log" ) -// MakeDeployStatus get deploy status from store -func (m *Mercury) MakeDeployStatus(ctx context.Context, appname, entryname string, strategyInfos []strategy.Info) error { +// GetDeployStatus get deploy status from store +func (m *Mercury) GetDeployStatus(ctx context.Context, appname, entryname string) (map[string]int, error) { // 手动加 / 防止不精确 key := filepath.Join(workloadDeployPrefix, appname, entryname) + "/" resp, err := m.Get(ctx, key, clientv3.WithPrefix(), clientv3.WithKeysOnly()) if err != nil { - return err + return nil, err } if resp.Count == 0 { log.Warnf(ctx, "[MakeDeployStatus] Deploy status not found %s.%s", appname, entryname) } - if err = m.doGetDeployStatus(ctx, resp, strategyInfos); err != nil { - return err + + deployCount := m.doGetDeployStatus(ctx, resp) + + processingCount, err := m.doLoadProcessing(ctx, appname, entryname) + if err != nil { + return nil, err + } + + // node count: deploy count + processing count + nodeCount := map[string]int{} + for node, count := range deployCount { + nodeCount[node] = count } - return m.doLoadProcessing(ctx, appname, entryname, strategyInfos) + for node, count := range processingCount { + nodeCount[node] += count + } + + return nodeCount, nil } -func (m *Mercury) doGetDeployStatus(_ context.Context, resp *clientv3.GetResponse, strategyInfos []strategy.Info) error { +// doGetDeployStatus returns how many workload have been deployed on each node +func (m *Mercury) doGetDeployStatus(_ context.Context, resp *clientv3.GetResponse) map[string]int { nodesCount := map[string]int{} for _, ev := range resp.Kvs { key := string(ev.Key) @@ -41,6 +55,5 @@ func (m *Mercury) doGetDeployStatus(_ context.Context, resp *clientv3.GetRespons nodesCount[nodename]++ } - setCount(nodesCount, strategyInfos) - return nil + return nodesCount } diff --git a/store/etcdv3/deploy_test.go b/store/etcdv3/deploy_test.go index e12e5007b..43d110b25 100644 --- a/store/etcdv3/deploy_test.go +++ b/store/etcdv3/deploy_test.go @@ -5,7 +5,6 @@ import ( "path/filepath" "testing" - "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" "github.com/stretchr/testify/assert" @@ -19,14 +18,11 @@ func TestDeploy(t *testing.T) { Entrypoint: &types.Entrypoint{Name: "entry"}, ProcessIdent: "abc", } - sis := []strategy.Info{ - {Nodename: "node"}, - } // no workload deployed - err := m.MakeDeployStatus(ctx, opts.Name, opts.Entrypoint.Name, sis) + nodeCount, err := m.GetDeployStatus(ctx, opts.Name, opts.Entrypoint.Name) assert.NoError(t, err) - assert.Equal(t, len(sis), 1) + assert.Equal(t, len(nodeCount), 0) // have workloads key := filepath.Join(workloadDeployPrefix, opts.Name, opts.Entrypoint.Name, "node", "id1") _, err = m.Put(ctx, key, "") @@ -34,9 +30,7 @@ func TestDeploy(t *testing.T) { key = filepath.Join(workloadDeployPrefix, opts.Name, opts.Entrypoint.Name, "node", "id2") _, err = m.Put(ctx, key, "") assert.NoError(t, err) - err = m.MakeDeployStatus(ctx, opts.Name, opts.Entrypoint.Name, sis) + nodeCount, err = m.GetDeployStatus(ctx, opts.Name, opts.Entrypoint.Name) assert.NoError(t, err) - assert.Equal(t, len(sis), 1) - assert.Equal(t, sis[0].Nodename, "node") - assert.Equal(t, sis[0].Count, 2) + assert.Equal(t, nodeCount["node"], 2) } diff --git a/store/etcdv3/helper.go b/store/etcdv3/helper.go index dc96b1d6e..cfdb406c1 100644 --- a/store/etcdv3/helper.go +++ b/store/etcdv3/helper.go @@ -2,8 +2,6 @@ package etcdv3 import ( "strings" - - "github.com/projecteru2/core/strategy" ) func parseStatusKey(key string) (string, string, string, string) { @@ -11,11 +9,3 @@ func parseStatusKey(key string) (string, string, string, string) { l := len(parts) return parts[l-4], parts[l-3], parts[l-2], parts[l-1] } - -func setCount(nodesCount map[string]int, strategyInfos []strategy.Info) { - for i, strategyInfo := range strategyInfos { - if v, ok := nodesCount[strategyInfo.Nodename]; ok { - strategyInfos[i].Count += v - } - } -} diff --git a/store/etcdv3/helper_test.go b/store/etcdv3/helper_test.go index 9d0270e5c..c9d6719dd 100644 --- a/store/etcdv3/helper_test.go +++ b/store/etcdv3/helper_test.go @@ -3,8 +3,6 @@ package etcdv3 import ( "testing" - "github.com/projecteru2/core/strategy" - "github.com/stretchr/testify/assert" ) @@ -16,17 +14,3 @@ func TestParseStatusKey(t *testing.T) { assert.Equal(t, p3, "node") assert.Equal(t, p4, "id") } - -func TestSetCount(t *testing.T) { - nodesCount := map[string]int{ - "n1": 1, - "n2": 2, - } - sis := []strategy.Info{ - {Nodename: "n1"}, - {Nodename: "n2"}, - } - setCount(nodesCount, sis) - assert.Equal(t, sis[0].Count, 1) - assert.Equal(t, sis[1].Count, 2) -} diff --git a/store/etcdv3/node.go b/store/etcdv3/node.go index d662f46e5..70eb65b6a 100644 --- a/store/etcdv3/node.go +++ b/store/etcdv3/node.go @@ -5,7 +5,6 @@ import ( "encoding/json" "fmt" "path/filepath" - "strconv" "strings" "github.com/pkg/errors" @@ -16,7 +15,6 @@ import ( enginefactory "github.com/projecteru2/core/engine/factory" "github.com/projecteru2/core/engine/fake" "github.com/projecteru2/core/log" - "github.com/projecteru2/core/metrics" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" ) @@ -33,52 +31,7 @@ func (m *Mercury) AddNode(ctx context.Context, opts *types.AddNodeOptions) (*typ return nil, err } - // 尝试加载的客户端 - // 会自动判断是否是支持的 url - client, err := enginefactory.GetEngine(ctx, m.config, opts.Nodename, opts.Endpoint, opts.Ca, opts.Cert, opts.Key) - if err != nil { - return nil, err - } - - // 判断这货是不是活着的 - info, err := client.Info(ctx) - if err != nil { - return nil, err - } - // 更新默认值 - if opts.CPU == 0 { - opts.CPU = info.NCPU - } - if opts.Memory == 0 { - opts.Memory = info.MemTotal * 8 / 10 // use 80% real memory - } - if opts.Storage == 0 { - opts.Storage = info.StorageTotal * 8 / 10 - } - if opts.Share == 0 { - opts.Share = m.config.Scheduler.ShareBase - } - if opts.Volume == nil { - opts.Volume = types.VolumeMap{} - } - // 设置 numa 的内存默认值,如果没有的话,按照 numa node 个数均分 - if len(opts.Numa) > 0 { - nodeIDs := map[string]struct{}{} - for _, nodeID := range opts.Numa { - nodeIDs[nodeID] = struct{}{} - } - perNodeMemory := opts.Memory / int64(len(nodeIDs)) - if opts.NumaMemory == nil { - opts.NumaMemory = types.NUMAMemory{} - } - for nodeID := range nodeIDs { - if _, ok := opts.NumaMemory[nodeID]; !ok { - opts.NumaMemory[nodeID] = perNodeMemory - } - } - } - - return m.doAddNode(ctx, opts.Nodename, opts.Endpoint, opts.Podname, opts.Ca, opts.Cert, opts.Key, opts.CPU, opts.Share, opts.Memory, opts.Storage, opts.Labels, opts.Numa, opts.NumaMemory, opts.Volume) + return m.doAddNode(ctx, opts.Nodename, opts.Endpoint, opts.Podname, opts.Ca, opts.Cert, opts.Key, opts.Labels) } // RemoveNode delete a node @@ -174,20 +127,6 @@ func (m *Mercury) UpdateNodes(ctx context.Context, nodes ...*types.Node) error { return nil } -// UpdateNodeResource update cpu and memory on a node, either add or subtract -func (m *Mercury) UpdateNodeResource(ctx context.Context, node *types.Node, resource *types.ResourceMeta, action string) error { - switch action { - case types.ActionIncr: - node.RecycleResources(resource) - case types.ActionDecr: - node.PreserveResources(resource) - default: - return types.ErrUnknownControlType - } - go metrics.Client.SendNodeInfo(node.Metrics()) - return m.UpdateNodes(ctx, node) -} - // SetNodeStatus sets status for a node, value will expire after ttl seconds // ttl < 0 means to delete node status // this is heartbeat of node @@ -294,7 +233,7 @@ func (m *Mercury) makeClient(ctx context.Context, node *types.Node) (client engi return enginefactory.GetEngine(ctx, m.config, node.Name, node.Endpoint, data[0], data[1], data[2]) } -func (m *Mercury) doAddNode(ctx context.Context, name, endpoint, podname, ca, cert, key string, cpu, share int, memory, storage int64, labels map[string]string, numa types.NUMA, numaMemory types.NUMAMemory, volumemap types.VolumeMap) (*types.Node, error) { +func (m *Mercury) doAddNode(ctx context.Context, name, endpoint, podname, ca, cert, key string, labels map[string]string) (*types.Node, error) { data := map[string]string{} // 如果有tls的证书需要保存就保存一下 if ca != "" { @@ -307,28 +246,12 @@ func (m *Mercury) doAddNode(ctx context.Context, name, endpoint, podname, ca, ce data[fmt.Sprintf(nodeKeyKey, name)] = key } - cpumap := types.CPUMap{} - for i := 0; i < cpu; i++ { - cpumap[strconv.Itoa(i)] = int64(share) - } - node := &types.Node{ NodeMeta: types.NodeMeta{ - Name: name, - Endpoint: endpoint, - Podname: podname, - CPU: cpumap, - MemCap: memory, - StorageCap: storage, - Volume: volumemap, - InitCPU: cpumap, - InitMemCap: memory, - InitStorageCap: storage, - InitNUMAMemory: numaMemory, - InitVolume: volumemap, - Labels: labels, - NUMA: numa, - NUMAMemory: numaMemory, + Name: name, + Endpoint: endpoint, + Podname: podname, + Labels: labels, }, Available: true, Bypass: false, @@ -351,7 +274,7 @@ func (m *Mercury) doAddNode(ctx context.Context, name, endpoint, podname, ca, ce return nil, types.ErrTxnConditionFailed } - go metrics.Client.SendNodeInfo(node.Metrics()) + // TODO: go metrics.Client.SendNodeInfo(node.Metrics()) return node, nil } @@ -380,7 +303,6 @@ func (m *Mercury) doGetNodes(ctx context.Context, kvs []*mvccpb.KeyValue, labels if err := json.Unmarshal(ev.Value, node); err != nil { return nil, err } - node.Init() node.Engine = &fake.Engine{DefaultErr: types.ErrNilEngine} if utils.FilterWorkload(node.Labels, labels) { allNodes = append(allNodes, node) diff --git a/store/etcdv3/node_test.go b/store/etcdv3/node_test.go index 33a925d04..0ea1c03d1 100644 --- a/store/etcdv3/node_test.go +++ b/store/etcdv3/node_test.go @@ -23,55 +23,29 @@ func TestAddNode(t *testing.T) { assert.NoError(t, err) _, err = m.AddPod(ctx, "numapod", "test") assert.NoError(t, err) - cpu := 1 - share := 100 - memory := int64(100) - storage := int64(100) - m.config.Scheduler.ShareBase = 100 labels := map[string]string{"test": "1"} - // wrong endpoint - ctx1, cancel := context.WithTimeout(ctx, time.Second) - defer cancel() - _, err = m.AddNode(ctx1, &types.AddNodeOptions{Nodename: nodename, Endpoint: "abc", Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) - assert.Error(t, err) - - // wrong because engine not mocked - ctx2, cancel := context.WithTimeout(ctx, time.Second) - defer cancel() - _, err = m.AddNode(ctx2, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) - assert.Error(t, err) endpoint = "mock://fakeengine" // wrong no pod - _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: "abc", CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: "abc", Labels: labels}) assert.Error(t, err) // AddNode - node, err := m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) + node, err := m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, Labels: labels}) assert.NoError(t, err) assert.Equal(t, node.Name, nodename) - assert.Equal(t, node.CPU["0"], int64(100)) // add again and failed - _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, Labels: labels}) assert.Error(t, err) - // AddNode with numa - nodeWithNuma, err := m.AddNode(ctx, &types.AddNodeOptions{Nodename: "nodewithnuma", Endpoint: endpoint, Podname: "numapod", CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels, Numa: types.NUMA{"1": "n1", "2": "n2"}}) - assert.NoError(t, err) - assert.Equal(t, nodeWithNuma.Name, "nodewithnuma") - assert.Equal(t, len(nodeWithNuma.NUMAMemory), 2) - assert.Equal(t, nodeWithNuma.NUMAMemory["n1"], int64(50)) // Addnode again will failed - _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, Labels: labels}) assert.Error(t, err) // Check etcd has node data key := fmt.Sprintf(nodeInfoKey, nodename) _, err = m.GetOne(ctx, key) assert.NoError(t, err) // AddNode with mocked engine and default value - node2, err := m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename2, Endpoint: endpoint, Podname: podname, Labels: labels}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename2, Endpoint: endpoint, Podname: podname, Labels: labels}) assert.NoError(t, err) - assert.Equal(t, node2.CPU["0"], int64(100)) - assert.Equal(t, len(node2.CPU), 100) - assert.Equal(t, node2.MemCap, int64(85899345920)) // with tls ca := `-----BEGIN CERTIFICATE----- MIIC7TCCAdWgAwIBAgIJAM8uLRZf9jttMA0GCSqGSIb3DQEBCwUAMA0xCzAJBgNV @@ -138,7 +112,7 @@ RdCPRPt513WozkJZZAjUSP2U nodename3 := "nodename3" endpoint3 := "tcp://path" m.config.CertPath = "/tmp" - node3, err := m.doAddNode(ctx, nodename3, endpoint3, podname, ca, cert, certkey, cpu, share, memory, storage, labels, nil, nil, nil) + node3, err := m.doAddNode(ctx, nodename3, endpoint3, podname, ca, cert, certkey, labels) assert.NoError(t, err) _, err = m.makeClient(ctx, node3) assert.Error(t, err) @@ -151,7 +125,7 @@ RdCPRPt513WozkJZZAjUSP2U func TestRemoveNode(t *testing.T) { m := NewMercury(t) ctx := context.Background() - node, err := m.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", 100, 100, 100000, 100000, nil, nil, nil, nil) + node, err := m.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", nil) assert.NoError(t, err) assert.Equal(t, node.Name, "test") assert.NoError(t, m.RemoveNode(ctx, nil)) @@ -161,7 +135,7 @@ func TestRemoveNode(t *testing.T) { func TestGetNode(t *testing.T) { m := NewMercury(t) ctx := context.Background() - node, err := m.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", 100, 100, 100000, 100000, nil, nil, nil, nil) + node, err := m.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", nil) assert.NoError(t, err) assert.Equal(t, node.Name, "test") _, err = m.GetNode(ctx, "wtf") @@ -174,7 +148,7 @@ func TestGetNode(t *testing.T) { func TestGetNodesByPod(t *testing.T) { m := NewMercury(t) ctx := context.Background() - node, err := m.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", 100, 100, 100000, 100000, map[string]string{"x": "y"}, nil, nil, nil) + node, err := m.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", nil) assert.NoError(t, err) assert.Equal(t, node.Name, "test") ns, err := m.GetNodesByPod(ctx, "wtf", nil, false) @@ -196,7 +170,7 @@ func TestGetNodesByPod(t *testing.T) { func TestUpdateNode(t *testing.T) { m := NewMercury(t) ctx := context.Background() - node, err := m.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", 100, 100, 100000, 100000, map[string]string{"x": "y"}, nil, nil, nil) + node, err := m.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", nil) assert.NoError(t, err) assert.Equal(t, node.Name, "test") fakeNode := &types.Node{ @@ -215,17 +189,6 @@ func TestUpdateNode(t *testing.T) { assert.NoError(t, m.UpdateNodes(ctx, node)) } -func TestUpdateNodeResource(t *testing.T) { - m := NewMercury(t) - ctx := context.Background() - 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, "wtf")) - assert.NoError(t, m.UpdateNodeResource(ctx, node, &types.ResourceMeta{CPU: map[string]int64{"0": 100}}, types.ActionIncr)) - assert.NoError(t, m.UpdateNodeResource(ctx, node, &types.ResourceMeta{CPU: map[string]int64{"0": 100}}, types.ActionDecr)) -} - func TestExtractNodename(t *testing.T) { assert := assert.New(t) assert.Equal(extractNodename("/nodestatus/testname"), "testname") diff --git a/store/etcdv3/pod_test.go b/store/etcdv3/pod_test.go index 5c637cfdd..4e1181c40 100644 --- a/store/etcdv3/pod_test.go +++ b/store/etcdv3/pod_test.go @@ -30,7 +30,7 @@ func TestPod(t *testing.T) { assert.Equal(t, len(pods), 1) assert.Equal(t, pods[0].Name, podname) - _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: "test", Endpoint: "mock://", Podname: podname, CPU: 10, Share: 100, Memory: 1000, Storage: 1000}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: "test", Endpoint: "mock://", Podname: podname}) assert.NoError(t, err) err = m.RemovePod(ctx, podname) assert.Error(t, err) diff --git a/store/etcdv3/processing.go b/store/etcdv3/processing.go index 43e9662c7..7314ddc0b 100644 --- a/store/etcdv3/processing.go +++ b/store/etcdv3/processing.go @@ -8,13 +8,15 @@ import ( "strings" "github.com/projecteru2/core/log" - "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" - "github.com/sanity-io/litter" clientv3 "go.etcd.io/etcd/client/v3" ) +func (m *Mercury) getProcessingKey(processing *types.Processing) string { + return filepath.Join(workloadProcessingPrefix, processing.Appname, processing.Entryname, processing.Nodename, processing.Ident) +} + // CreateProcessing save processing status in etcd func (m *Mercury) CreateProcessing(ctx context.Context, processing *types.Processing, count int) error { _, err := m.Create(ctx, m.getProcessingKey(processing), fmt.Sprintf("%d", count)) @@ -27,22 +29,18 @@ func (m *Mercury) DeleteProcessing(ctx context.Context, processing *types.Proces return err } -func (m *Mercury) getProcessingKey(processing *types.Processing) string { - return filepath.Join(workloadProcessingPrefix, processing.Appname, processing.Entryname, processing.Nodename, processing.Ident) -} - -func (m *Mercury) doLoadProcessing(ctx context.Context, appname, entryname string, strategyInfos []strategy.Info) error { - // 显式的加 / 保证 prefix 一致性 +func (m *Mercury) doLoadProcessing(ctx context.Context, appname, entryname string) (map[string]int, error) { + nodesCount := map[string]int{} + // 显式地加 / 保证 prefix 一致性 processingKey := filepath.Join(workloadProcessingPrefix, appname, entryname) + "/" resp, err := m.Get(ctx, processingKey, clientv3.WithPrefix()) if err != nil { - return err + return nil, err } if resp.Count == 0 { - return nil + return nodesCount, nil } - nodesCount := map[string]int{} for _, ev := range resp.Kvs { key := string(ev.Key) parts := strings.Split(key, "/") @@ -58,8 +56,6 @@ func (m *Mercury) doLoadProcessing(ctx context.Context, appname, entryname strin } nodesCount[nodename] += count } - - log.Debug(ctx, "[doLoadProcessing] Processing result: ", litter.Options{Compact: true}.Sdump(nodesCount)) - setCount(nodesCount, strategyInfos) - return nil + log.Debugf(ctx, "[doLoadProcessing] Processing result: %+v", nodesCount) + return nodesCount, nil } diff --git a/store/etcdv3/processing_test.go b/store/etcdv3/processing_test.go index f7cb64575..9539adde4 100644 --- a/store/etcdv3/processing_test.go +++ b/store/etcdv3/processing_test.go @@ -4,7 +4,6 @@ import ( "context" "testing" - "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" "github.com/stretchr/testify/assert" @@ -26,10 +25,9 @@ func TestProcessing(t *testing.T) { assert.Error(t, m.CreateProcessing(ctx, processing, 10)) assert.NoError(t, m.AddWorkload(ctx, &types.Workload{Name: "a_b_c"}, processing)) - sis := []strategy.Info{{Nodename: "node"}} - err := m.doLoadProcessing(ctx, processing.Appname, processing.Entryname, sis) + nodeCount, err := m.doLoadProcessing(ctx, processing.Appname, processing.Entryname) assert.NoError(t, err) - assert.Equal(t, sis[0].Count, 9) + assert.Equal(t, nodeCount["node"], 9) // delete assert.NoError(t, m.DeleteProcessing(ctx, processing)) } diff --git a/store/etcdv3/workload_test.go b/store/etcdv3/workload_test.go index a57b89e16..09ee6b048 100644 --- a/store/etcdv3/workload_test.go +++ b/store/etcdv3/workload_test.go @@ -95,7 +95,7 @@ func TestGetWorkload(t *testing.T) { // create pod node _, err = m.AddPod(ctx, podname, "") assert.NoError(t, err) - _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname, CPU: 10, Share: 100, Memory: 1000, Storage: 1000}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname}) assert.NoError(t, err) // success _, err = m.GetWorkload(ctx, ID) @@ -124,7 +124,7 @@ func TestGetWorkloadStatus(t *testing.T) { // add success _, err = m.AddPod(ctx, podname, "") assert.NoError(t, err) - _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname, CPU: 10, Share: 100, Memory: 1000, Storage: 1000}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname}) assert.NoError(t, err) c, err := m.GetWorkloadStatus(ctx, ID) assert.Nil(t, c) @@ -194,7 +194,7 @@ func TestListWorkloads(t *testing.T) { assert.NoError(t, err) _, err = m.AddPod(ctx, podname, "") assert.NoError(t, err) - _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname, CPU: 10, Share: 100, Memory: 1000, Storage: 1000}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname}) assert.NoError(t, err) // no labels cs, err = m.ListWorkloads(ctx, "", "a", "b", 1, nil) @@ -230,7 +230,7 @@ func TestListNodeWorkloads(t *testing.T) { assert.NoError(t, err) _, err = m.AddPod(ctx, podname, "") assert.NoError(t, err) - _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname, CPU: 10, Share: 100, Memory: 1000, Storage: 1000}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname}) assert.NoError(t, err) // no labels cs, err = m.ListNodeWorkloads(ctx, nodename, nil) diff --git a/store/mocks/Store.go b/store/mocks/Store.go index e01aa70eb..6d6d1380f 100644 --- a/store/mocks/Store.go +++ b/store/mocks/Store.go @@ -8,8 +8,6 @@ import ( 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" @@ -154,6 +152,29 @@ func (_m *Store) GetAllPods(ctx context.Context) ([]*types.Pod, error) { return r0, r1 } +// GetDeployStatus provides a mock function with given fields: ctx, appname, entryname +func (_m *Store) GetDeployStatus(ctx context.Context, appname string, entryname string) (map[string]int, error) { + ret := _m.Called(ctx, appname, entryname) + + var r0 map[string]int + if rf, ok := ret.Get(0).(func(context.Context, string, string) map[string]int); ok { + r0 = rf(ctx, appname, entryname) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(map[string]int) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(context.Context, string, string) error); ok { + r1 = rf(ctx, appname, entryname) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // GetNode provides a mock function with given fields: ctx, nodename func (_m *Store) GetNode(ctx context.Context, nodename string) (*types.Node, error) { ret := _m.Called(ctx, nodename) @@ -384,20 +405,6 @@ func (_m *Store) ListWorkloads(ctx context.Context, appname string, entrypoint s return r0, r1 } -// MakeDeployStatus provides a mock function with given fields: ctx, appname, entryname, sis -func (_m *Store) MakeDeployStatus(ctx context.Context, appname string, entryname string, sis []strategy.Info) error { - ret := _m.Called(ctx, appname, entryname, sis) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, string, string, []strategy.Info) error); ok { - r0 = rf(ctx, appname, entryname, sis) - } else { - r0 = ret.Error(0) - } - - return r0 -} - // NodeStatusStream provides a mock function with given fields: ctx func (_m *Store) NodeStatusStream(ctx context.Context) chan *types.NodeStatus { ret := _m.Called(ctx) @@ -571,20 +578,6 @@ func (_m *Store) StartEphemeral(ctx context.Context, path string, heartbeat time return r0, r1, r2 } -// 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.ResourceMeta, string) error); ok { - r0 = rf(ctx, node, resource, action) - } else { - r0 = ret.Error(0) - } - - return r0 -} - // UpdateNodes provides a mock function with given fields: _a0, _a1 func (_m *Store) UpdateNodes(_a0 context.Context, _a1 ...*types.Node) error { _va := make([]interface{}, len(_a1)) diff --git a/store/redis/deploy.go b/store/redis/deploy.go index b657e2e1b..9fdec333f 100644 --- a/store/redis/deploy.go +++ b/store/redis/deploy.go @@ -6,38 +6,47 @@ import ( "strings" "github.com/projecteru2/core/log" - "github.com/projecteru2/core/strategy" ) -// MakeDeployStatus get deploy status from store -func (r *Rediaron) MakeDeployStatus(ctx context.Context, appname, entryname string, strategyInfos []strategy.Info) error { +// GetDeployStatus . +func (r *Rediaron) GetDeployStatus(ctx context.Context, appname, entryname string) (map[string]int, error) { // 手动加 / 防止不精确 key := filepath.Join(workloadDeployPrefix, appname, entryname) + "/*" data, err := r.getByKeyPattern(ctx, key, 0) if err != nil { - return err + return nil, err } if len(data) == 0 { log.Warnf(ctx, "[MakeDeployStatus] Deploy status not found %s.%s", appname, entryname) } - if err = r.doGetDeployStatus(ctx, data, strategyInfos); err != nil { - return err + + deployCount := r.doGetDeployStatus(ctx, data) + + processingCount, err := r.doLoadProcessing(ctx, appname, entryname) + if err != nil { + return nil, err } - return r.doLoadProcessing(ctx, appname, entryname, strategyInfos) + + // node count: deploy count + processing count + nodeCount := map[string]int{} + for node, count := range deployCount { + nodeCount[node] = count + } + for node, count := range processingCount { + nodeCount[node] += count + } + + return nodeCount, nil } -func (r *Rediaron) doGetDeployStatus(_ context.Context, data map[string]string, strategyInfos []strategy.Info) error { +// doGetDeployStatus returns how many workload have been deployed on each node +func (r *Rediaron) doGetDeployStatus(_ context.Context, data map[string]string) map[string]int { nodesCount := map[string]int{} for key := range data { parts := strings.Split(key, "/") nodename := parts[len(parts)-2] - if _, ok := nodesCount[nodename]; !ok { - nodesCount[nodename] = 1 - continue - } nodesCount[nodename]++ } - setCount(nodesCount, strategyInfos) - return nil + return nodesCount } diff --git a/store/redis/deploy_test.go b/store/redis/deploy_test.go index f21ea1d1b..9194e6dbe 100644 --- a/store/redis/deploy_test.go +++ b/store/redis/deploy_test.go @@ -4,7 +4,6 @@ import ( "context" "path/filepath" - "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" ) @@ -15,14 +14,11 @@ func (s *RediaronTestSuite) TestDeploy() { Entrypoint: &types.Entrypoint{Name: "entry"}, ProcessIdent: "abc", } - sis := []strategy.Info{ - {Nodename: "node"}, - } // no workload deployed - err := s.rediaron.MakeDeployStatus(ctx, opts.Name, opts.Entrypoint.Name, sis) + nodeCount, err := s.rediaron.GetDeployStatus(ctx, opts.Name, opts.Entrypoint.Name) s.NoError(err) - s.Equal(len(sis), 1) + s.Equal(len(nodeCount), 0) // have workloads key := filepath.Join(workloadDeployPrefix, opts.Name, opts.Entrypoint.Name, "node", "id1") _, err = s.rediaron.cli.Set(ctx, key, "", 0).Result() @@ -31,9 +27,7 @@ func (s *RediaronTestSuite) TestDeploy() { s.NoError(err) _, err = s.rediaron.cli.Set(ctx, key, "", 0).Result() s.NoError(err) - err = s.rediaron.MakeDeployStatus(ctx, opts.Name, opts.Entrypoint.Name, sis) + nodeCount, err = s.rediaron.GetDeployStatus(ctx, opts.Name, opts.Entrypoint.Name) s.NoError(err) - s.Equal(len(sis), 1) - s.Equal(sis[0].Nodename, "node") - s.Equal(sis[0].Count, 2) + s.Equal(nodeCount["node"], 2) } diff --git a/store/redis/helper.go b/store/redis/helper.go index a34311760..d5bfb3a1a 100644 --- a/store/redis/helper.go +++ b/store/redis/helper.go @@ -3,8 +3,6 @@ package redis import ( "context" "strings" - - "github.com/projecteru2/core/strategy" ) // extracts node name from key @@ -20,14 +18,6 @@ func parseStatusKey(key string) (string, string, string, string) { return parts[l-4], parts[l-3], parts[l-2], parts[l-1] } -func setCount(nodesCount map[string]int, strategyInfos []strategy.Info) { - for i, strategyInfo := range strategyInfos { - if v, ok := nodesCount[strategyInfo.Nodename]; ok { - strategyInfos[i].Count += v - } - } -} - // getByKeyPattern gets key-value pairs that key matches pattern func (r *Rediaron) getByKeyPattern(ctx context.Context, pattern string, limit int64) (map[string]string, error) { var ( diff --git a/store/redis/helper_test.go b/store/redis/helper_test.go index 4b76644f8..fcaa07a61 100644 --- a/store/redis/helper_test.go +++ b/store/redis/helper_test.go @@ -3,8 +3,6 @@ package redis import ( "testing" - "github.com/projecteru2/core/strategy" - "github.com/stretchr/testify/assert" ) @@ -16,17 +14,3 @@ func TestParseStatusKey(t *testing.T) { assert.Equal(t, p3, "node") assert.Equal(t, p4, "id") } - -func TestSetCount(t *testing.T) { - nodesCount := map[string]int{ - "n1": 1, - "n2": 2, - } - sis := []strategy.Info{ - {Nodename: "n1"}, - {Nodename: "n2"}, - } - setCount(nodesCount, sis) - assert.Equal(t, sis[0].Count, 1) - assert.Equal(t, sis[1].Count, 2) -} diff --git a/store/redis/node.go b/store/redis/node.go index 5061c63b1..c60c21e7f 100644 --- a/store/redis/node.go +++ b/store/redis/node.go @@ -5,7 +5,6 @@ import ( "encoding/json" "fmt" "path/filepath" - "strconv" "strings" "time" @@ -13,7 +12,6 @@ import ( enginefactory "github.com/projecteru2/core/engine/factory" "github.com/projecteru2/core/engine/fake" "github.com/projecteru2/core/log" - "github.com/projecteru2/core/metrics" "github.com/projecteru2/core/types" "github.com/projecteru2/core/utils" @@ -32,52 +30,7 @@ func (r *Rediaron) AddNode(ctx context.Context, opts *types.AddNodeOptions) (*ty return nil, err } - // 尝试加载的客户端 - // 会自动判断是否是支持的 url - client, err := enginefactory.GetEngine(ctx, r.config, opts.Nodename, opts.Endpoint, opts.Ca, opts.Cert, opts.Key) - if err != nil { - return nil, err - } - - // 判断这货是不是活着的 - info, err := client.Info(ctx) - if err != nil { - return nil, err - } - // 更新默认值 - if opts.CPU == 0 { - opts.CPU = info.NCPU - } - if opts.Memory == 0 { - opts.Memory = info.MemTotal * 8 / 10 // use 80% real memory - } - if opts.Storage == 0 { - opts.Storage = info.StorageTotal * 8 / 10 - } - if opts.Share == 0 { - opts.Share = r.config.Scheduler.ShareBase - } - if opts.Volume == nil { - opts.Volume = types.VolumeMap{} - } - // 设置 numa 的内存默认值,如果没有的话,按照 numa node 个数均分 - if len(opts.Numa) > 0 { - nodeIDs := map[string]struct{}{} - for _, nodeID := range opts.Numa { - nodeIDs[nodeID] = struct{}{} - } - perNodeMemory := opts.Memory / int64(len(nodeIDs)) - if opts.NumaMemory == nil { - opts.NumaMemory = types.NUMAMemory{} - } - for nodeID := range nodeIDs { - if _, ok := opts.NumaMemory[nodeID]; !ok { - opts.NumaMemory[nodeID] = perNodeMemory - } - } - } - - return r.doAddNode(ctx, opts.Nodename, opts.Endpoint, opts.Podname, opts.Ca, opts.Cert, opts.Key, opts.CPU, opts.Share, opts.Memory, opts.Storage, opts.Labels, opts.Numa, opts.NumaMemory, opts.Volume) + return r.doAddNode(ctx, opts.Nodename, opts.Endpoint, opts.Podname, opts.Ca, opts.Cert, opts.Key, opts.Labels) } // RemoveNode delete a node @@ -165,20 +118,6 @@ func (r *Rediaron) UpdateNodes(ctx context.Context, nodes ...*types.Node) error return errors.WithStack(r.BatchPut(ctx, data)) } -// UpdateNodeResource update cpu and memory on a node, either add or subtract -func (r *Rediaron) UpdateNodeResource(ctx context.Context, node *types.Node, resource *types.ResourceMeta, action string) error { - switch action { - case types.ActionIncr: - node.RecycleResources(resource) - case types.ActionDecr: - node.PreserveResources(resource) - default: - return types.ErrUnknownControlType - } - go metrics.Client.SendNodeInfo(node.Metrics()) - return r.UpdateNodes(ctx, node) -} - // SetNodeStatus sets status for a node, value will expire after ttl seconds // ttl < 0 means delete node status // this is heartbeat of node @@ -281,7 +220,7 @@ func (r *Rediaron) makeClient(ctx context.Context, node *types.Node) (client eng return client, nil } -func (r *Rediaron) doAddNode(ctx context.Context, name, endpoint, podname, ca, cert, key string, cpu, share int, memory, storage int64, labels map[string]string, numa types.NUMA, numaMemory types.NUMAMemory, volumemap types.VolumeMap) (*types.Node, error) { +func (r *Rediaron) doAddNode(ctx context.Context, name, endpoint, podname, ca, cert, key string, labels map[string]string) (*types.Node, error) { data := map[string]string{} // 如果有tls的证书需要保存就保存一下 if ca != "" { @@ -294,28 +233,12 @@ func (r *Rediaron) doAddNode(ctx context.Context, name, endpoint, podname, ca, c data[fmt.Sprintf(nodeKeyKey, name)] = key } - cpumap := types.CPUMap{} - for i := 0; i < cpu; i++ { - cpumap[strconv.Itoa(i)] = int64(share) - } - node := &types.Node{ NodeMeta: types.NodeMeta{ - Name: name, - Endpoint: endpoint, - Podname: podname, - CPU: cpumap, - MemCap: memory, - StorageCap: storage, - Volume: volumemap, - InitCPU: cpumap, - InitMemCap: memory, - InitStorageCap: storage, - InitNUMAMemory: numaMemory, - InitVolume: volumemap, - Labels: labels, - NUMA: numa, - NUMAMemory: numaMemory, + Name: name, + Endpoint: endpoint, + Podname: podname, + Labels: labels, }, Available: true, } @@ -334,7 +257,7 @@ func (r *Rediaron) doAddNode(ctx context.Context, name, endpoint, podname, ca, c return nil, err } - go metrics.Client.SendNodeInfo(node.Metrics()) + // TODO: go metrics.Client.SendNodeInfo(node.Metrics()) return node, nil } @@ -363,7 +286,6 @@ func (r *Rediaron) doGetNodes(ctx context.Context, kvs map[string]string, labels if err := json.Unmarshal([]byte(value), node); err != nil { return nil, err } - node.Init() node.Engine = &fake.Engine{DefaultErr: types.ErrNilEngine} if utils.FilterWorkload(node.Labels, labels) { allNodes = append(allNodes, node) @@ -376,7 +298,7 @@ func (r *Rediaron) doGetNodes(ctx context.Context, kvs map[string]string, labels for _, n := range allNodes { node := n pool.Go(ctx, func() { - if _, err := r.GetNodeStatus(ctx, node.Name); err != nil && !errors.Is(err, types.ErrBadCount) { + if _, err := r.GetNodeStatus(ctx, node.Name); err != nil && !isRedisNoKeyError(err) { log.Errorf(ctx, "[doGetNodes] failed to get node status of %v, err: %v", node.Name, err) } else { node.Available = err == nil diff --git a/store/redis/node_test.go b/store/redis/node_test.go index dfdb8e5f9..b27c3003c 100644 --- a/store/redis/node_test.go +++ b/store/redis/node_test.go @@ -2,7 +2,6 @@ package redis import ( "context" - "fmt" "path/filepath" "time" @@ -11,64 +10,14 @@ import ( func (s *RediaronTestSuite) TestAddNode() { ctx := context.Background() - nodename := "testnode" - nodename2 := "testnode2" - endpoint := "tcp://128.0.0.1:2376" podname := "testpod" _, err := s.rediaron.AddPod(ctx, podname, "test") s.NoError(err) _, err = s.rediaron.AddPod(ctx, "numapod", "test") s.NoError(err) - cpu := 1 - share := 100 - memory := int64(100) - storage := int64(100) s.rediaron.config.Scheduler.ShareBase = 100 labels := map[string]string{"test": "1"} - // wrong endpoint - ctx1, cancel := context.WithTimeout(ctx, time.Second) - defer cancel() - _, err = s.rediaron.AddNode(ctx1, &types.AddNodeOptions{Nodename: nodename, Endpoint: "abc", Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) - s.Error(err) - - // wrong because engine not mocked - ctx2, cancel := context.WithTimeout(ctx, time.Second) - defer cancel() - _, err = s.rediaron.AddNode(ctx2, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) - s.Error(err) - endpoint = "mock://fakeengine" - - // wrong no pod - _, err = s.rediaron.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: "abc", CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) - s.Error(err) - // AddNode - node, err := s.rediaron.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) - s.NoError(err) - s.Equal(node.Name, nodename) - s.Equal(node.CPU["0"], int64(100)) - // add again and failed - _, err = s.rediaron.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) - s.Error(err) - // AddNode with numa - nodeWithNuma, err := s.rediaron.AddNode(ctx, &types.AddNodeOptions{Nodename: "nodewithnuma", Endpoint: endpoint, Podname: "numapod", CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels, Numa: types.NUMA{"1": "n1", "2": "n2"}}) - s.NoError(err) - s.Equal(nodeWithNuma.Name, "nodewithnuma") - s.Equal(len(nodeWithNuma.NUMAMemory), 2) - s.Equal(nodeWithNuma.NUMAMemory["n1"], int64(50)) - // Addnode again will failed - _, err = s.rediaron.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: endpoint, Podname: podname, CPU: cpu, Share: share, Memory: memory, Storage: storage, Labels: labels}) - s.Error(err) - // Check etcd has node data - key := fmt.Sprintf(nodeInfoKey, nodename) - _, err = s.rediaron.GetOne(ctx, key) - s.NoError(err) - // AddNode with mocked engine and default value - node2, err := s.rediaron.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename2, Endpoint: endpoint, Podname: podname, Labels: labels}) - s.NoError(err) - s.Equal(node2.CPU["0"], int64(100)) - s.Equal(len(node2.CPU), 100) - s.Equal(node2.MemCap, int64(85899345920)) // with tls ca := `-----BEGIN CERTIFICATE----- MIIC7TCCAdWgAwIBAgIJAM8uLRZf9jttMA0GCSqGSIb3DQEBCwUAMA0xCzAJBgNV @@ -135,7 +84,7 @@ RdCPRPt513WozkJZZAjUSP2U nodename3 := "nodename3" endpoint3 := "tcp://path" s.rediaron.config.CertPath = "/tmp" - node3, err := s.rediaron.doAddNode(ctx, nodename3, endpoint3, podname, ca, cert, certkey, cpu, share, memory, storage, labels, nil, nil, nil) + node3, err := s.rediaron.doAddNode(ctx, nodename3, endpoint3, podname, ca, cert, certkey, labels) s.NoError(err) _, err = s.rediaron.makeClient(ctx, node3) s.Error(err) @@ -147,7 +96,7 @@ RdCPRPt513WozkJZZAjUSP2U func (s *RediaronTestSuite) TestRemoveNode() { ctx := context.Background() - node, err := s.rediaron.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", 100, 100, 100000, 100000, nil, nil, nil, nil) + node, err := s.rediaron.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", nil) s.NoError(err) s.Equal(node.Name, "test") s.NoError(s.rediaron.RemoveNode(ctx, nil)) @@ -156,7 +105,7 @@ func (s *RediaronTestSuite) TestRemoveNode() { func (s *RediaronTestSuite) TestGetNode() { ctx := context.Background() - node, err := s.rediaron.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", 100, 100, 100000, 100000, nil, nil, nil, nil) + node, err := s.rediaron.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", nil) s.NoError(err) s.Equal(node.Name, "test") _, err = s.rediaron.GetNode(ctx, "wtf") @@ -168,7 +117,7 @@ func (s *RediaronTestSuite) TestGetNode() { func (s *RediaronTestSuite) TestGetNodesByPod() { ctx := context.Background() - node, err := s.rediaron.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", 100, 100, 100000, 100000, map[string]string{"x": "y"}, nil, nil, nil) + node, err := s.rediaron.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", map[string]string{"x": "y"}) s.NoError(err) s.Equal(node.Name, "test") ns, err := s.rediaron.GetNodesByPod(ctx, "wtf", nil, false) @@ -189,7 +138,7 @@ func (s *RediaronTestSuite) TestGetNodesByPod() { func (s *RediaronTestSuite) TestUpdateNode() { ctx := context.Background() - node, err := s.rediaron.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", 100, 100, 100000, 100000, map[string]string{"x": "y"}, nil, nil, nil) + node, err := s.rediaron.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", map[string]string{"x": "y"}) s.NoError(err) s.Equal(node.Name, "test") fakeNode := &types.Node{ @@ -208,12 +157,9 @@ func (s *RediaronTestSuite) TestUpdateNode() { func (s *RediaronTestSuite) TestUpdateNodeResource() { ctx := context.Background() - node, err := s.rediaron.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) + node, err := s.rediaron.doAddNode(ctx, "test", "mock://", "testpod", "", "", "", map[string]string{"x": "y"}) s.NoError(err) s.Equal(node.Name, "test") - s.Error(s.rediaron.UpdateNodeResource(ctx, node, nil, "wtf")) - s.NoError(s.rediaron.UpdateNodeResource(ctx, node, &types.ResourceMeta{CPU: map[string]int64{"0": 100}}, types.ActionIncr)) - s.NoError(s.rediaron.UpdateNodeResource(ctx, node, &types.ResourceMeta{CPU: map[string]int64{"0": 100}}, types.ActionDecr)) } func (s *RediaronTestSuite) TestExtractNodename() { diff --git a/store/redis/pod_test.go b/store/redis/pod_test.go index 260a5108e..585945496 100644 --- a/store/redis/pod_test.go +++ b/store/redis/pod_test.go @@ -26,7 +26,7 @@ func (s *RediaronTestSuite) TestPod() { s.Equal(len(pods), 1) s.Equal(pods[0].Name, podname) - _, err = s.rediaron.AddNode(ctx, &types.AddNodeOptions{Nodename: "test", Endpoint: "mock://", Podname: podname, CPU: 10, Share: 100, Memory: 1000, Storage: 1000}) + _, err = s.rediaron.AddNode(ctx, &types.AddNodeOptions{Nodename: "test", Endpoint: "mock://", Podname: podname}) s.NoError(err) err = s.rediaron.RemovePod(ctx, podname) s.Error(err) diff --git a/store/redis/processing.go b/store/redis/processing.go index 033a28278..bf7ea8a81 100644 --- a/store/redis/processing.go +++ b/store/redis/processing.go @@ -7,12 +7,13 @@ import ( "strings" "github.com/projecteru2/core/log" - "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" - - "github.com/sanity-io/litter" ) +func (r *Rediaron) getProcessingKey(processing *types.Processing) string { + return filepath.Join(workloadProcessingPrefix, processing.Appname, processing.Entryname, processing.Nodename, processing.Ident) +} + // CreateProcessing save processing status in etcd func (r *Rediaron) CreateProcessing(ctx context.Context, processing *types.Processing, count int) error { processingKey := r.getProcessingKey(processing) @@ -24,22 +25,19 @@ func (r *Rediaron) DeleteProcessing(ctx context.Context, processing *types.Proce return r.BatchDelete(ctx, []string{r.getProcessingKey(processing)}) } -func (r *Rediaron) getProcessingKey(processing *types.Processing) string { - return filepath.Join(workloadProcessingPrefix, processing.Appname, processing.Entryname, processing.Nodename, processing.Ident) -} - -func (r *Rediaron) doLoadProcessing(ctx context.Context, appname, entryname string, strategyInfos []strategy.Info) error { - // 显式的加 / 保证 prefix 一致性 +// doLoadProcessing returns how many workloads are `processing` on each node +func (r *Rediaron) doLoadProcessing(ctx context.Context, appname, entryname string) (map[string]int, error) { + nodesCount := map[string]int{} + // 显式地加 / 保证 prefix 一致性 processingKey := filepath.Join(workloadProcessingPrefix, appname, entryname) + "/*" data, err := r.getByKeyPattern(ctx, processingKey, 0) if err != nil { - return err + return nil, err } if len(data) == 0 { - return nil + return nodesCount, nil } - nodesCount := map[string]int{} for k, v := range data { parts := strings.Split(k, "/") nodename := parts[len(parts)-2] @@ -55,8 +53,6 @@ func (r *Rediaron) doLoadProcessing(ctx context.Context, appname, entryname stri nodesCount[nodename] += count } - log.Debug(ctx, "[doLoadProcessing] Processing result:") - litter.Dump(nodesCount) - setCount(nodesCount, strategyInfos) - return nil + log.Debug(ctx, "[doLoadProcessing] Processing result: %+v", nodesCount) + return nodesCount, nil } diff --git a/store/redis/processing_test.go b/store/redis/processing_test.go index abf92ce65..0a85df541 100644 --- a/store/redis/processing_test.go +++ b/store/redis/processing_test.go @@ -3,7 +3,6 @@ package redis import ( "context" - "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" ) @@ -22,10 +21,9 @@ func (s *RediaronTestSuite) TestProcessing() { s.Error(s.rediaron.CreateProcessing(ctx, processing, 10)) s.NoError(s.rediaron.AddWorkload(ctx, &types.Workload{Name: "a_b_c"}, processing)) - sis := []strategy.Info{{Nodename: "node"}} - err := s.rediaron.doLoadProcessing(ctx, processing.Appname, processing.Entryname, sis) + nodeCount, err := s.rediaron.doLoadProcessing(ctx, processing.Appname, processing.Entryname) s.NoError(err) - s.Equal(sis[0].Count, 9) + s.Equal(nodeCount["node"], 9) // delete s.NoError(s.rediaron.DeleteProcessing(ctx, processing)) } diff --git a/store/redis/workload_test.go b/store/redis/workload_test.go index 1e4b38dd1..8fd3de6bc 100644 --- a/store/redis/workload_test.go +++ b/store/redis/workload_test.go @@ -89,10 +89,6 @@ func (s *RediaronTestSuite) TestGetWorkload() { Nodename: nodename, Endpoint: "mock://", Podname: podname, - CPU: 10, - Share: 100, - Memory: 1000, - Storage: 1000, }) s.NoError(err) // success @@ -125,10 +121,6 @@ func (s *RediaronTestSuite) TestGetWorkloadStatus() { Nodename: nodename, Endpoint: "mock://", Podname: podname, - CPU: 10, - Share: 100, - Memory: 1000, - Storage: 1000, }) s.NoError(err) c, err := s.rediaron.GetWorkloadStatus(ctx, id) @@ -202,10 +194,6 @@ func (s *RediaronTestSuite) TestListWorkloads() { Nodename: nodename, Endpoint: "mock://", Podname: podname, - CPU: 10, - Share: 100, - Memory: 1000, - Storage: 1000, }) s.NoError(err) // no labels @@ -242,7 +230,7 @@ func (s *RediaronTestSuite) TestListNodeWorkloads() { s.NoError(err) _, err = m.AddPod(ctx, podname, "") s.NoError(err) - _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname, CPU: 10, Share: 100, Memory: 1000, Storage: 1000}) + _, err = m.AddNode(ctx, &types.AddNodeOptions{Nodename: nodename, Endpoint: "mock://", Podname: podname}) s.NoError(err) // no labels cs, err = m.ListNodeWorkloads(ctx, nodename, nil) diff --git a/store/store.go b/store/store.go index fe76c6836..99a3aea1a 100644 --- a/store/store.go +++ b/store/store.go @@ -8,7 +8,6 @@ import ( "github.com/projecteru2/core/lock" "github.com/projecteru2/core/store/etcdv3" "github.com/projecteru2/core/store/redis" - "github.com/projecteru2/core/strategy" "github.com/projecteru2/core/types" ) @@ -34,7 +33,6 @@ 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, resource *types.ResourceMeta, action string) error SetNodeStatus(ctx context.Context, node *types.Node, ttl int64) error GetNodeStatus(ctx context.Context, nodename string) (*types.NodeStatus, error) NodeStatusStream(ctx context.Context) chan *types.NodeStatus @@ -52,7 +50,7 @@ type Store interface { WorkloadStatusStream(ctx context.Context, appname, entrypoint, nodename string, labels map[string]string) chan *types.WorkloadStatus // deploy status - MakeDeployStatus(ctx context.Context, appname, entryname string, sis []strategy.Info) error + GetDeployStatus(ctx context.Context, appname, entryname string) (map[string]int, error) // processing status CreateProcessing(ctx context.Context, process *types.Processing, count int) error diff --git a/strategy/communism_test.go b/strategy/communism_test.go index 80dc98b99..26400dbad 100644 --- a/strategy/communism_test.go +++ b/strategy/communism_test.go @@ -3,13 +3,9 @@ package strategy import ( "context" "fmt" - "math/rand" "sort" "testing" - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/utils" - "github.com/stretchr/testify/assert" ) @@ -46,36 +42,38 @@ func TestCommunismPlan(t *testing.T) { assert.ElementsMatch(t, []int{12, 13, 15, 17}, getFinalStatus(r, nodes)) } -func randomDeployStatus(scheduleInfos []resourcetypes.ScheduleInfo, maxDeployed int) (sis []Info) { - s := rand.NewSource(int64(1024)) - r := rand.New(s) - for range scheduleInfos { - sis = append(sis, Info{ - Capacity: maxDeployed, - Count: r.Intn(maxDeployed), - }) - } - return -} - -func Benchmark_CommunismPlan(b *testing.B) { - b.StopTimer() - var count = 10000 - var maxDeployed = 1024 - var volTotal = maxDeployed * count - var need = volTotal - 1 - // Simulate `count` nodes with difference deploy status, each one can deploy `maxDeployed` workloads - // and then we deploy `need` workloads - for i := 0; i < b.N; i++ { - // 24 core, 128G memory, 10 pieces per core - t := utils.GenerateScheduleInfos(count, 1, 1, 0, 10) - hugePod := randomDeployStatus(t, maxDeployed) - b.StartTimer() - _, err := CommunismPlan(context.TODO(), hugePod, need, 100, 0) - b.StopTimer() - assert.NoError(b, err) - } -} +// +//func randomDeployStatus(scheduleInfos []resourcetypes.ScheduleInfo, maxDeployed int) (sis []Info) { +// s := rand.NewSource(int64(1024)) +// r := rand.New(s) +// for range scheduleInfos { +// sis = append(sis, Info{ +// Capacity: maxDeployed, +// Count: r.Intn(maxDeployed), +// }) +// } +// return +//} + +// +//func Benchmark_CommunismPlan(b *testing.B) { +// b.StopTimer() +// var count = 10000 +// var maxDeployed = 1024 +// var volTotal = maxDeployed * count +// var need = volTotal - 1 +// // Simulate `count` nodes with difference deploy status, each one can deploy `maxDeployed` workloads +// // and then we deploy `need` workloads +// for i := 0; i < b.N; i++ { +// // 24 core, 128G memory, 10 pieces per core +// t := utils.GenerateScheduleInfos(count, 1, 1, 0, 10) +// hugePod := randomDeployStatus(t, maxDeployed) +// b.StartTimer() +// _, err := CommunismPlan(context.TODO(), hugePod, need, 100, 0) +// b.StopTimer() +// assert.NoError(b, err) +// } +//} func genNodesByCapCount(caps, counts []int) (infos []Info) { for i := range caps { infos = append(infos, Info{ diff --git a/strategy/strategy.go b/strategy/strategy.go index daeab2d93..f93134f4d 100644 --- a/strategy/strategy.go +++ b/strategy/strategy.go @@ -2,14 +2,11 @@ package strategy import ( "context" - "math" + + "github.com/pkg/errors" "github.com/projecteru2/core/log" - resourcetypes "github.com/projecteru2/core/resources/types" "github.com/projecteru2/core/types" - "github.com/projecteru2/core/utils" - - "github.com/pkg/errors" ) const ( @@ -36,17 +33,17 @@ var Plans = map[string]strategyFunc{ type strategyFunc = func(_ context.Context, _ []Info, need, total, limit int) (map[string]int, error) // Deploy . -func Deploy(ctx context.Context, opts *types.DeployOptions, strategyInfos []Info, total int) (map[string]int, error) { - deployMethod, ok := Plans[opts.DeployStrategy] +func Deploy(ctx context.Context, strategy string, count, nodesLimit int, strategyInfos []Info, total int) (map[string]int, error) { + deployMethod, ok := Plans[strategy] if !ok { return nil, errors.WithStack(types.ErrBadDeployStrategy) } - if opts.Count <= 0 { + if count <= 0 { return nil, errors.WithStack(types.ErrBadCount) } - log.Debugf(ctx, "[strategy.Deploy] infos %+v, need %d, total %d, limit %d", strategyInfos, opts.Count, total, opts.NodesLimit) - return deployMethod(ctx, strategyInfos, opts.Count, total, opts.NodesLimit) + log.Debugf(ctx, "[strategy.Deploy] infos %+v, need %d, total %d, limit %d", strategyInfos, count, total, nodesLimit) + return deployMethod(ctx, strategyInfos, count, total, nodesLimit) } // Info . @@ -59,25 +56,3 @@ type Info struct { Capacity int Count int } - -// NewInfos . -// 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 { - capacity := math.MaxInt64 - for _, plan := range plans { - capacity = utils.Min(capacity, plan.Capacity()[nodename]) - } - if capacity <= 0 { - continue - } - - strategyInfos = append(strategyInfos, Info{ - Nodename: nodename, - Rate: resourceRequests.MainRateOnNode(*node), - Usage: resourceRequests.MainUsageOnNode(*node), - Capacity: capacity, - }) - } - return -} diff --git a/strategy/strategy_test.go b/strategy/strategy_test.go index 615440297..8e04dde98 100644 --- a/strategy/strategy_test.go +++ b/strategy/strategy_test.go @@ -4,9 +4,6 @@ import ( "context" "testing" - "github.com/projecteru2/core/resources" - resourcetypes "github.com/projecteru2/core/resources/types" - resourcetypesmocks "github.com/projecteru2/core/resources/types/mocks" "github.com/projecteru2/core/types" "github.com/stretchr/testify/assert" @@ -43,24 +40,11 @@ func TestDeploy(t *testing.T) { Count: 1, NodesLimit: 3, } - _, err := Deploy(context.TODO(), opts, nil, 2) + _, err := Deploy(context.TODO(), opts.DeployStrategy, opts.Count, opts.NodesLimit, nil, 2) opts.DeployStrategy = "AUTO" Plans["test"] = func(_ context.Context, _ []Info, _, _, _ int) (map[string]int, error) { return nil, nil } - _, err = Deploy(context.TODO(), opts, nil, 2) + _, err = Deploy(context.TODO(), opts.DeployStrategy, opts.Count, opts.NodesLimit, nil, 2) assert.Error(t, err) } - -func TestNewInfos(t *testing.T) { - rrs, err := resources.MakeRequests(types.ResourceOptions{}) - assert.Nil(t, err) - nodeMap := map[string]*types.Node{ - "node1": {}, - "node2": {}, - } - mockPlan := &resourcetypesmocks.ResourcePlans{} - mockPlan.On("Capacity").Return(map[string]int{"node1": 1}) - plans := []resourcetypes.ResourcePlans{mockPlan} - NewInfos(rrs, nodeMap, plans) -} diff --git a/types/config.go b/types/config.go index 7bce93d1a..bab2265c1 100644 --- a/types/config.go +++ b/types/config.go @@ -29,6 +29,9 @@ type Config struct { MaxConcurrency int64 `yaml:"max_concurrency" default:"20"` // concurrently call single runtime in the same time Store string `yaml:"store" default:"etcd"` // store type + ResourcePluginsDir string `yaml:"resource_plugins_dir" default:"/etc/eru/resource_plugins"` // resource plugins path + ResourcePluginsTimeout time.Duration `yaml:"resource_plugins_timeout" default:"30s"` // timeout for calling resource plugins + Auth AuthConfig `yaml:"auth"` // grpc auth GRPCConfig GRPCConfig `yaml:"grpc"` // grpc config @@ -118,8 +121,9 @@ type LogConfig struct { // SchedConfig holds scheduler config type SchedConfig struct { - MaxShare int `yaml:"maxshare" required:"true" default:"-1"` // comlpex scheduler use maxshare - ShareBase int `yaml:"sharebase" required:"true" default:"100"` // how many pieces for one core + MaxShare int `yaml:"maxshare" required:"true" default:"-1"` // comlpex scheduler use maxshare + ShareBase int `yaml:"sharebase" required:"true" default:"100"` // how many pieces for one core + MaxDeployCount int `yaml:"max_deploy_count" required:"false" default:"10000"` // max deploy count of each node } // AuthConfig contains authorization information for connecting to a Registry diff --git a/types/errors.go b/types/errors.go index b0e4294bb..e036a3f2c 100644 --- a/types/errors.go +++ b/types/errors.go @@ -115,6 +115,9 @@ var ( ErrNoFilesToSend = errors.New("no files to send") ErrNoFilesToCopy = errors.New("no files to copy") + + ErrInvalidEngineArgs = errors.New("invalid engine args") + ErrGetMostIdleNodeFailed = errors.New("get most idle node failed") ) type detailedErr struct { @@ -136,3 +139,23 @@ func (d detailedErr) Unwrap() error { func NewDetailedErr(err error, details interface{}) error { return detailedErr{err: err, details: details} } + +// CombinedErr can combine multiple errors into one +type CombinedErr struct { + ErrMap map[string]error +} + +// Error . +func (e *CombinedErr) Error() string { + return fmt.Sprintf("%+v", e.ErrMap) +} + +// Append . +func (e *CombinedErr) Append(key string, err error) { + e.ErrMap[key] = err +} + +// NewCombinedErr . +func NewCombinedErr() *CombinedErr { + return &CombinedErr{ErrMap: map[string]error{}} +} diff --git a/types/message.go b/types/message.go index ffa4900e2..96d50ed24 100644 --- a/types/message.go +++ b/types/message.go @@ -79,7 +79,8 @@ type ControlWorkloadMessage struct { // CreateWorkloadMessage for create message type CreateWorkloadMessage struct { - ResourceMeta + EngineArgs EngineArgs + ResourceArgs map[string]WorkloadResourceArgs Podname string Nodename string WorkloadID string diff --git a/types/node.go b/types/node.go index f978712d8..f55dd81d6 100644 --- a/types/node.go +++ b/types/node.go @@ -3,7 +3,6 @@ package types import ( "context" "encoding/json" - "math" "github.com/pkg/errors" @@ -30,18 +29,6 @@ type NodeMeta struct { Podname string `json:"podname"` Labels map[string]string `json:"labels"` - CPU CPUMap `json:"cpu"` - Volume VolumeMap `json:"volume"` - NUMA NUMA `json:"numa"` - NUMAMemory NUMAMemory `json:"numa_memory"` - MemCap int64 `json:"memcap"` - StorageCap int64 `json:"storage_cap"` - InitCPU CPUMap `json:"init_cpu"` - InitMemCap int64 `json:"init_memcap"` - InitStorageCap int64 `json:"init_storage_cap"` - InitNUMAMemory NUMAMemory `json:"init_numa_memory"` - InitVolume VolumeMap `json:"init_volume"` - Ca string `json:"-"` Cert string `json:"-"` Key string `json:"-"` @@ -61,8 +48,8 @@ type Node struct { NodeMeta NodeInfo string `json:"-"` - CPUUsed float64 `json:"cpuused"` - VolumeUsed int64 `json:"volumeused"` + ResourceCapacity map[string]NodeResourceArgs `json:"resource_capacity,omitempty"` + ResourceUsage map[string]NodeResourceArgs `json:"resource_usage,omitempty"` // Bypass if bypass is true, it will not participate in future scheduling Bypass bool `json:"bypass,omitempty"` @@ -70,16 +57,6 @@ type Node struct { Engine engine.API `json:"-"` } -// Init . -func (n *Node) Init() { - if n.Volume == nil { - n.Volume = VolumeMap{} - } - if n.InitVolume == nil { - n.InitVolume = VolumeMap{} - } -} - // Info show node info func (n *Node) Info(ctx context.Context) (err error) { info, err := n.Engine.Info(ctx) @@ -97,132 +74,6 @@ func (n *Node) Info(ctx context.Context) (err error) { return nil } -// SetCPUUsed set cpuusage -func (n *Node) SetCPUUsed(quota float64, action string) { - switch action { - case IncrUsage: - n.CPUUsed = Round(n.CPUUsed + quota) - case DecrUsage: - n.CPUUsed = Round(n.CPUUsed - quota) - default: - } -} - -// SetVolumeUsed . -func (n *Node) SetVolumeUsed(cost int64, action string) { - switch action { - case IncrUsage: - n.VolumeUsed += cost - case DecrUsage: - n.VolumeUsed -= cost - default: - } -} - -// GetNUMANode get numa node -func (n *Node) GetNUMANode(cpu CPUMap) string { - nodeID := "" - for cpuID := range cpu { - if memoryNode, ok := n.NUMA[cpuID]; ok { - if nodeID == "" { - nodeID = memoryNode - } else if nodeID != memoryNode { // 如果跨 NODE 了,让系统决定 nodeID - nodeID = "" - } - } - } - return nodeID -} - -// IncrNUMANodeMemory set numa node memory -func (n *Node) IncrNUMANodeMemory(nodeID string, memory int64) { - if _, ok := n.NUMAMemory[nodeID]; ok { - n.NUMAMemory[nodeID] += memory - } -} - -// DecrNUMANodeMemory set numa node memory -func (n *Node) DecrNUMANodeMemory(nodeID string, memory int64) { - if _, ok := n.NUMAMemory[nodeID]; ok { - n.NUMAMemory[nodeID] -= memory - } -} - -// StorageUsage calculates node's storage usage ratio. -func (n *Node) StorageUsage() float64 { - switch { - case n.InitStorageCap <= 0: - return 1.0 - default: - return 1.0 - float64(n.StorageCap)/float64(n.InitStorageCap) - } -} - -// StorageUsed calculates node's storage usage value. -func (n *Node) StorageUsed() int64 { - switch { - case n.InitStorageCap <= 0: - return 0 - default: - return n.InitStorageCap - n.StorageCap - } -} - -// AvailableStorage calculates available value. -func (n *Node) AvailableStorage() int64 { - switch { - case n.InitStorageCap <= 0: - return math.MaxInt64 - default: - return n.StorageCap - } -} - -// ResourceUsages . -func (n *Node) ResourceUsages() map[ResourceType]float64 { - res := map[ResourceType]float64{ - ResourceCPU: n.CPUUsed / float64(len(n.InitCPU)), - ResourceMemory: 1.0 - float64(n.MemCap)/float64(n.InitMemCap), - ResourceStorage: n.StorageUsage(), - ResourceVolume: float64(n.VolumeUsed) / float64(n.InitVolume.Total()), - } - for k, v := range res { - if v > 1.0 { - res[k] = 1.0 - } - if v < 0 { - res[k] = 0 - } - } - return res -} - -// RecycleResources . -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 . -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) - } -} - // IsDown returns if the node is marked as down. func (n *Node) IsDown() bool { // If `bypass` is true, then even if the node is still healthy, the node will be regarded as `down`. @@ -232,47 +83,28 @@ func (n *Node) IsDown() bool { // NodeMetrics used for metrics collecting type NodeMetrics struct { - Name string - Podname string - Memory float64 - MemoryUsed float64 - Storage float64 - StorageUsed float64 - CPUUsed float64 - CPU CPUMap + Name string + Podname string + ResourceCapacity map[string]NodeResourceArgs + ResourceUsage map[string]NodeResourceArgs } // Metrics reports metrics value func (n *Node) Metrics() *NodeMetrics { - nc := CPUMap{} - for k, v := range n.CPU { - nc[k] = v - } return &NodeMetrics{ - Name: n.Name, - Podname: n.Podname, - Memory: float64(n.MemCap), - MemoryUsed: float64(n.InitMemCap - n.MemCap), - Storage: float64(n.StorageCap), - StorageUsed: float64(n.InitStorageCap - n.StorageCap), - CPUUsed: n.CPUUsed, - CPU: nc, + Name: n.Name, + Podname: n.Podname, + // TODO: resource args } } // NodeResource for node check type NodeResource struct { - Name string - CPU CPUMap - MemCap int64 - StorageCap int64 - CPUPercent float64 - MemoryPercent float64 - StoragePercent float64 - NUMAMemoryPercent map[string]float64 - VolumePercent float64 - Diffs []string - Workloads []*Workload + Name string + ResourceCapacity map[string]NodeResourceArgs + ResourceUsage map[string]NodeResourceArgs + Diffs []string + Workloads []*Workload } // NodeStatus wraps node status diff --git a/types/node_test.go b/types/node_test.go index e537ccb9c..22be757fc 100644 --- a/types/node_test.go +++ b/types/node_test.go @@ -2,8 +2,6 @@ package types import ( "context" - "math" - "reflect" "strings" "testing" @@ -14,178 +12,16 @@ import ( "github.com/stretchr/testify/mock" ) -func TestNode(t *testing.T) { +func TestNodeInfo(t *testing.T) { mockEngine := &enginemocks.API{} r := &enginetypes.Info{ID: "test"} mockEngine.On("Info", mock.Anything).Return(r, nil) node := &Node{} - assert.Nil(t, node.Volume) - assert.Nil(t, node.InitVolume) - node.Init() - assert.Equal(t, node.Volume, VolumeMap{}) - assert.Equal(t, node.InitVolume, VolumeMap{}) ctx := context.Background() node.Engine = mockEngine err := node.Info(ctx) assert.NoError(t, err) assert.True(t, strings.Contains(node.NodeInfo, "test")) - - node.CPUUsed = 0.0 - node.SetCPUUsed(1.0, IncrUsage) - assert.Equal(t, node.CPUUsed, 1.0) - node.SetCPUUsed(1.0, DecrUsage) - assert.Equal(t, node.CPUUsed, 0.0) - - node.SetVolumeUsed(100, IncrUsage) - assert.Equal(t, node.VolumeUsed, int64(100)) - node.SetVolumeUsed(10, DecrUsage) - assert.Equal(t, node.VolumeUsed, int64(90)) -} - -func TestGetNUMANode(t *testing.T) { - node := &Node{ - NodeMeta: NodeMeta{NUMA: NUMA{"1": "node1", "2": "node2", "3": "node1", "4": "node2"}}, - } - cpu := CPUMap{"1": 100, "2": 100} - nodeID := node.GetNUMANode(cpu) - assert.Equal(t, nodeID, "") - cpu = CPUMap{"1": 100, "3": 100} - nodeID = node.GetNUMANode(cpu) - assert.Equal(t, nodeID, "node1") - cpu = nil - nodeID = node.GetNUMANode(cpu) - assert.Equal(t, nodeID, "") -} - -func TestSetNUMANodeMemory(t *testing.T) { - node := &Node{ - NodeMeta: NodeMeta{NUMAMemory: NUMAMemory{"n1": 100}}, - } - // incr - node.IncrNUMANodeMemory("n1", 1) - assert.Len(t, node.NUMAMemory, 1) - assert.Equal(t, node.NUMAMemory["n1"], int64(101)) - // decr - node.DecrNUMANodeMemory("n1", 1) - assert.Len(t, node.NUMAMemory, 1) - assert.Equal(t, node.NUMAMemory["n1"], int64(100)) -} - -func TestStorage(t *testing.T) { - node := &Node{ - NodeMeta: NodeMeta{InitStorageCap: 0}, - } - assert.Equal(t, node.StorageUsage(), 1.0) - assert.Equal(t, node.StorageUsed(), int64(0)) - assert.Equal(t, node.AvailableStorage(), int64(math.MaxInt64)) - node.InitStorageCap = 2 - node.StorageCap = 1 - assert.Equal(t, node.StorageUsage(), 0.5) - assert.Equal(t, node.StorageUsed(), int64(1)) - assert.Equal(t, node.AvailableStorage(), int64(1)) -} - -func TestNodeUsage(t *testing.T) { - node := Node{ - NodeMeta: NodeMeta{ - CPU: CPUMap{"0": 100, "1": 50}, - InitCPU: CPUMap{"0": 200, "1": 200}, - Volume: VolumeMap{"/data1": 1000, "/data2": 2000}, - InitVolume: VolumeMap{"/data1": 1000, "/data2": 2500}, - MemCap: 1, - InitMemCap: 100, - StorageCap: 0, - InitStorageCap: 2, - }, - CPUUsed: 1.5, - VolumeUsed: 500, - } - usages := node.ResourceUsages() - assert.EqualValues(t, 1, usages[ResourceStorage]) - assert.EqualValues(t, 0.99, usages[ResourceMemory]) - assert.EqualValues(t, 0.75, usages[ResourceCPU]) - assert.EqualValues(t, 500./3500., usages[ResourceVolume]) - - // negate resources - node = Node{ - NodeMeta: NodeMeta{ - CPU: CPUMap{"0": -100, "1": -50}, - InitCPU: CPUMap{"0": 200, "1": 200}, - Volume: VolumeMap{"/data1": -1000, "/data2": -2000}, - InitVolume: VolumeMap{"/data1": 1000, "/data2": 2500}, - MemCap: -1, - InitMemCap: 100, - StorageCap: -2, - InitStorageCap: 2, - }, - CPUUsed: 5.5, - VolumeUsed: 6500, - } - usages = node.ResourceUsages() - assert.EqualValues(t, 1, usages[ResourceStorage]) - assert.EqualValues(t, 1, usages[ResourceMemory]) - assert.EqualValues(t, 1, usages[ResourceCPU]) - assert.EqualValues(t, 1, usages[ResourceVolume]) -} - -func TestAddNodeOptions(t *testing.T) { - o := AddNodeOptions{ - Volume: VolumeMap{"/data1": 1, "/data2": 2}, - } - o.Normalize() - assert.EqualValues(t, 3, o.Storage) -} - -func TestNodeWithResource(t *testing.T) { - n := Node{ - NodeMeta: NodeMeta{ - 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}}), - NUMANode: "0", - } - 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) -} - -func TestNodeMeta(t *testing.T) { - meta := NodeMeta{ - CPU: CPUMap{ - "0": 100, - }, - Volume: VolumeMap{ - "/tmp": 1000, - }, - } - - co, err := meta.DeepCopy() - assert.Nil(t, err) - co.CPU["0"] = 0 - co.Volume["/tmp"]++ - assert.EqualValues(t, 100, meta.CPU["0"]) - assert.EqualValues(t, 1000, meta.Volume["/tmp"]) } diff --git a/types/options.go b/types/options.go index c5a9418e2..34dcbe457 100644 --- a/types/options.go +++ b/types/options.go @@ -11,7 +11,7 @@ import ( // DeployOptions is options for deploying type DeployOptions struct { - ResourceOpts ResourceOptions + ResourceOpts WorkloadResourceOpts Name string // Name of application Entrypoint *Entrypoint // entrypoint Podname string // Name of pod to deploy @@ -185,20 +185,14 @@ type ListNodesOptions struct { // AddNodeOptions for adding node type AddNodeOptions struct { - Nodename string - Endpoint string - Podname string - Ca string - Cert string - Key string - CPU int - Share int - Memory int64 - Storage int64 - Labels map[string]string - Numa NUMA - NumaMemory NUMAMemory - Volume VolumeMap + Nodename string + Endpoint string + Podname string + Ca string + Cert string + Key string + ResourceOpts NodeResourceOpts + Labels map[string]string } // Validate checks options @@ -212,52 +206,21 @@ func (o *AddNodeOptions) Validate() error { if o.Endpoint == "" { return errors.WithStack(ErrEmptyNodeEndpoint) } - if o.CPU < 0 { - return errors.WithStack(ErrNegativeCPU) - } - if o.Share < 0 { - return errors.WithStack(ErrNegativeShare) - } - if o.Memory < 0 { - return errors.WithStack(ErrNegativeMemory) - } - for _, m := range o.NumaMemory { - if m < 0 { - return errors.WithStack(ErrNegativeNUMAMemory) - } - } - for _, size := range o.Volume { - if size < 0 { - return errors.WithStack(ErrNegativeVolumeSize) - } - } - if o.Storage < 0 { - return errors.WithStack(ErrNegativeStorage) - } return nil } -// Normalize keeps options consistent -func (o *AddNodeOptions) Normalize() { - o.Storage += o.Volume.Total() -} - // SetNodeOptions for node set type SetNodeOptions struct { - Nodename string - Endpoint string - WorkloadsDown bool - DeltaCPU CPUMap - DeltaMemory int64 - DeltaStorage int64 - DeltaNUMAMemory map[string]int64 - DeltaVolume VolumeMap - NUMA map[string]string - Labels map[string]string - BypassOpt TriOptions - Ca string - Cert string - Key string + Nodename string + Endpoint string + WorkloadsDown bool + ResourceOpts NodeResourceOpts + Delta bool + Labels map[string]string + BypassOpt TriOptions + Ca string + Cert string + Key string } // Validate checks options @@ -268,16 +231,6 @@ func (o *SetNodeOptions) Validate() error { return nil } -// Normalize keeps options consistent -func (o *SetNodeOptions) Normalize(node *Node) { - o.DeltaStorage += o.DeltaVolume.Total() - for volID, size := range o.DeltaVolume { - if size == 0 { - o.DeltaStorage -= node.InitVolume[volID] - } - } -} - // ImageOptions wraps options for images // Prune is only used when remove image type ImageOptions struct { @@ -317,8 +270,7 @@ type ExecuteWorkloadOptions struct { // ReallocOptions . type ReallocOptions struct { ID string - CPUBindOpts TriOptions - ResourceOpts ResourceOptions + ResourceOpts WorkloadResourceOpts } // TriOptions . diff --git a/types/options_test.go b/types/options_test.go index 727f6f304..c9dce73a8 100644 --- a/types/options_test.go +++ b/types/options_test.go @@ -16,28 +16,12 @@ func TestParseTriOption(t *testing.T) { func TestSetNodeOptions(t *testing.T) { o := &SetNodeOptions{ - DeltaVolume: VolumeMap{"/data": 1, "/data2": 2}, - DeltaStorage: -10, + ResourceOpts: map[string]interface{}{}, } assert.Equal(t, ErrEmptyNodeName, errors.Unwrap(o.Validate())) o.Nodename = "nodename" assert.NoError(t, o.Validate()) - - o.Normalize(nil) - assert.EqualValues(t, -7, o.DeltaStorage) - - node := &Node{ - NodeMeta: NodeMeta{ - InitVolume: VolumeMap{"/data0": 100, "/data1": 3}, - }, - } - o = &SetNodeOptions{ - DeltaVolume: VolumeMap{"/data0": 0, "/data1": 10}, - DeltaStorage: 10, - } - o.Normalize(node) - assert.EqualValues(t, 10-100+10, o.DeltaStorage) } func TestDeployOptions(t *testing.T) { @@ -138,25 +122,6 @@ func TestValidatingAddNodeOptions(t *testing.T) { assert.Equal(ErrEmptyNodeEndpoint, errors.Unwrap(o.Validate())) o.Endpoint = "tcp://endpoint:2376" - o.CPU = -1 - o.Share = -1 - o.Memory = -1 - o.NumaMemory = NUMAMemory{"0": -1} - o.Volume = VolumeMap{"/data": -1} - o.Storage = -1 - - assert.Equal(ErrNegativeCPU, errors.Unwrap(o.Validate())) - o.CPU = 1 - assert.Equal(ErrNegativeShare, errors.Unwrap(o.Validate())) - o.Share = 100 - assert.Equal(ErrNegativeMemory, errors.Unwrap(o.Validate())) - o.Memory = 100 - assert.Equal(ErrNegativeNUMAMemory, errors.Unwrap(o.Validate())) - o.NumaMemory = nil - assert.Equal(ErrNegativeVolumeSize, errors.Unwrap(o.Validate())) - o.Volume = nil - assert.Equal(ErrNegativeStorage, errors.Unwrap(o.Validate())) - o.Storage = 1 assert.NoError(o.Validate()) } diff --git a/types/resource.go b/types/resource.go index 2354e68e6..472faef1d 100644 --- a/types/resource.go +++ b/types/resource.go @@ -2,251 +2,120 @@ package types import ( "encoding/json" - "sort" - - "github.com/pkg/errors" + "fmt" + "strconv" ) -// ResourceOptions for create/realloc/replace -type ResourceOptions struct { - CPUQuotaRequest float64 - CPUQuotaLimit float64 - CPUBind bool - CPU CPUMap - - MemoryRequest int64 - MemoryLimit int64 - - VolumeRequest VolumeBindings - VolumeLimit VolumeBindings - VolumeExist VolumePlan - - StorageRequest int64 - StorageLimit int64 -} - // ResourceMeta for messages and workload to store -type ResourceMeta struct { - CPUQuotaRequest float64 `json:"cpu_quota_request"` - CPUQuotaLimit float64 `json:"cpu_quota_limit"` - CPU ResourceMap `json:"cpu"` - NUMANode string `json:"numa_node"` - - MemoryRequest int64 `json:"memory_request"` - MemoryLimit int64 `json:"memory_limit"` - - VolumeRequest VolumeBindings `json:"volume_request"` - VolumeLimit VolumeBindings `json:"volume_limit"` - VolumePlanRequest VolumePlan `json:"volume_plan_request"` - VolumePlanLimit VolumePlan `json:"volume_plan_limit"` - VolumeChanged bool `json:"volume_changed"` - - StorageRequest int64 `json:"storage_request"` - StorageLimit int64 `json:"storage_limit"` -} - -// ResourceType . -type ResourceType int - -const ( - // ResourceCPU . - ResourceCPU ResourceType = 1 << iota - // ResourceCPUBind . - ResourceCPUBind - // ResourceMemory . - ResourceMemory - // ResourceVolume . - ResourceVolume - // ResourceScheduledVolume . - ResourceScheduledVolume - // ResourceStorage . - ResourceStorage -) - -var ( - // ResourceAll . - ResourceAll = ResourceStorage | ResourceMemory | ResourceCPU | ResourceVolume - // AllResourceTypes . - AllResourceTypes = [...]ResourceType{ResourceCPU, ResourceMemory, ResourceVolume, ResourceStorage} -) +type ResourceMeta map[string]WorkloadResourceArgs -// ResourceMap is cpu core map -// ResourceMap {["0"]10000, ["1"]10000} -type ResourceMap map[string]int64 +// RawParams . +type RawParams map[string]interface{} -// Total show total cpu -// Total quotas -func (c ResourceMap) Total() int64 { - var count int64 - for _, value := range c { - count += value - } - return count +// IsSet . +func (r RawParams) IsSet(key string) bool { + _, ok := r[key] + return ok } -// 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 - } - } +// Float64 . +func (r RawParams) Float64(key string) float64 { + res, _ := strconv.ParseFloat(fmt.Sprintf("%v", r[key]), 64) + return res } -// Sub decrease cpus -func (c ResourceMap) Sub(q ResourceMap) { - for label, value := range q { - if _, ok := c[label]; ok { - c[label] -= value - } +// Int64 . +func (r RawParams) Int64(key string) int64 { + if !r.IsSet(key) { + return 0 } -} - -// 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 + var str string + if f, ok := r[key].(float64); ok { + str = fmt.Sprintf("%.0f", f) + } else { + str = fmt.Sprintf("%v", r[key]) } - return + res, _ := strconv.ParseInt(str, 10, 64) + return res } -// 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}) +// String . +func (r RawParams) String(key string) string { + if !r.IsSet(key) { + return "" } - 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] - } + if str, ok := r[key].(string); ok { + return str } - return volumePlan + return "" } -// UnmarshalJSON . -func (p *VolumePlan) UnmarshalJSON(b []byte) (err error) { - if *p == nil { - *p = VolumePlan{} +// StringSlice . +func (r RawParams) StringSlice(key string) []string { + if !r.IsSet(key) { + return nil } - plan := map[string]VolumeMap{} - if err = json.Unmarshal(b, &plan); err != nil { - return errors.WithStack(err) + if s, ok := r[key].([]string); ok { + return s } - for volume, vmap := range plan { - vb, err := NewVolumeBinding(volume) - if err != nil { - return errors.WithStack(err) + res := []string{} + if s, ok := r[key].([]interface{}); ok { + for _, v := range s { + if str, ok := v.(string); ok { + res = append(res, str) + } else { + return nil + } } - (*p)[*vb] = vmap } - return + return res } -// MarshalJSON . -func (p VolumePlan) MarshalJSON() ([]byte, error) { - plan := map[string]VolumeMap{} - for vb, vmap := range p { - plan[vb.ToString(false)] = vmap +// OneOfStringSlice . +func (r RawParams) OneOfStringSlice(keys ...string) []string { + for _, key := range keys { + if res := r.StringSlice(key); len(res) > 0 { + return res + } } - bs, err := json.Marshal(plan) - return bs, errors.WithStack(err) + return nil } -// 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 +// Bool . +func (r RawParams) Bool(key string) bool { + return r.IsSet(key) } -// IntoVolumeMap Merge return one VolumeMap with all in VolumePlan added -func (p VolumePlan) IntoVolumeMap() VolumeMap { - volumeMap := VolumeMap{} - for _, v := range p { - volumeMap.Add(v) +// RawParams . +func (r RawParams) RawParams(key string) map[string]interface{} { + if !r.IsSet(key) { + return map[string]interface{}{} } - 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 - } + if m, ok := r[key].(map[string]interface{}); ok { + return m } - return + return map[string]interface{}{} } -// 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) - // 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 +// ConvertRawParamsToMap . +func ConvertRawParamsToMap[V any](r RawParams) map[string]V { + res := map[string]V{} + body, _ := json.Marshal(r) + _ = json.Unmarshal(body, &res) + return res } -// Merge . -func (p VolumePlan) Merge(p2 VolumePlan) { - for vb, vm := range p2 { - p[vb] = vm - } -} +// NodeResourceOpts . +type NodeResourceOpts RawParams -// FindAffinityPlan . -func (p VolumePlan) FindAffinityPlan(req VolumeBinding) (_ VolumeBinding, _ VolumeMap, found bool) { - for vb, vm := range p { - if vb.Source == req.Source && vb.Destination == req.Destination && vb.Flags == req.Flags { - return vb, vm, true - } - } - found = false - return -} +// NodeResourceArgs . +type NodeResourceArgs RawParams + +// WorkloadResourceOpts . +type WorkloadResourceOpts RawParams + +// WorkloadResourceArgs . +type WorkloadResourceArgs RawParams + +// EngineArgs . +type EngineArgs RawParams diff --git a/types/resource_test.go b/types/resource_test.go index 5040e58de..1cf583468 100644 --- a/types/resource_test.go +++ b/types/resource_test.go @@ -1,113 +1,41 @@ package types import ( - "encoding/json" "testing" "github.com/stretchr/testify/assert" ) -func TestCPUMap(t *testing.T) { - cpuMap := CPUMap{"0": 50, "1": 70} - total := cpuMap.Total() - assert.Equal(t, total, int64(120)) - - cpuMap.Add(CPUMap{"0": 20}) - assert.Equal(t, cpuMap["0"], int64(70)) - - cpuMap.Add(CPUMap{"3": 100}) - assert.Equal(t, cpuMap["3"], int64(100)) - - cpuMap.Sub(CPUMap{"1": 20}) - assert.Equal(t, cpuMap["1"], int64(50)) -} - -func TestNewVolumePlan(t *testing.T) { - plan := MakeVolumePlan( - MustToVolumeBindings([]string{"AUTO:/data0:rw:10", "AUTO:/data1:ro:20", "AUTO:/data2:rw:10"}), - []VolumeMap{ - {"/dir0": 10}, - {"/dir1": 10}, - {"/dir2": 20}, +func TestRawParams(t *testing.T) { + var r RawParams + + r = RawParams{ + "int64": 1, + "str-int": "1", + "float-int": 1.999999999999999999999, + "float64": 1.999999999999999999999, + "string": "string", + "string-slice": []string{"string", "string"}, + "bool": nil, + "raw-params": map[string]interface{}{ + "int64": 1, + "str-int": "1", + "float-int": 1.999999999999999999999, + "float64": 1.999999999999999999999, + "string": "string", + "string-slice": []string{"string", "string"}, + "bool": nil, }, - ) - assert.Equal(t, plan, VolumePlan{ - MustToVolumeBinding("AUTO:/data0:rw:10"): VolumeMap{"/dir0": 10}, - MustToVolumeBinding("AUTO:/data1:ro:20"): VolumeMap{"/dir2": 20}, - MustToVolumeBinding("AUTO:/data2:rw:10"): VolumeMap{"/dir1": 10}, - }) - - data := []byte(`{"AUTO:/data0:rw:10":{"/dir0":10},"AUTO:/data1:ro:20":{"/dir2":20},"AUTO:/data2:rw:10":{"/dir1":10}}`) - b, err := json.Marshal(plan) - assert.Nil(t, err) - assert.Equal(t, b, data) - - plan2 := VolumePlan{} - err = json.Unmarshal(data, &plan2) - assert.Nil(t, err) - assert.Equal(t, plan2, plan) - - plan3 := MakeVolumePlan( - MustToVolumeBindings([]string{"AUTO:/data3:rw:10"}), - []VolumeMap{ - {"/dir0": 10}, - }, - ) - plan.Merge(plan3) - assert.Equal(t, len(plan), 4) - assert.Equal(t, plan[MustToVolumeBinding("AUTO:/data3:rw:10")], VolumeMap{"/dir0": 10}) -} - -func TestVolumeMap(t *testing.T) { - volume := VolumeMap{"/data": 1000} - assert.Equal(t, volume.Total(), int64(1000)) - assert.Equal(t, volume.GetResourceID(), "/data") - assert.Equal(t, volume.GetRation(), int64(1000)) - - volume = VolumeMap{"/data": 1000, "/data1": 1000, "/data2": 1002} - initVolume := VolumeMap{"/data": 1000, "/data1": 1001, "/data2": 1001} - used, unused := volume.SplitByUsed(initVolume) - assert.Equal(t, used, VolumeMap{"/data1": 1000, "/data2": 1002}) - assert.Equal(t, unused, VolumeMap{"/data": 1000}) -} - -func TestVolumePlan(t *testing.T) { - plan := VolumePlan{ - MustToVolumeBinding("AUTO:/data0:rw:100"): VolumeMap{"/dir0": 100}, - MustToVolumeBinding("AUTO:/data1:ro:2000"): VolumeMap{"/dir1": 2000}, } - assert.Equal(t, plan.IntoVolumeMap(), VolumeMap{"/dir0": 100, "/dir1": 2000}) - literal := map[string]map[string]int64{ - "AUTO:/data0:rw:100": {"/dir0": 100}, - "AUTO:/data1:ro:2000": {"/dir1": 2000}, - } - assert.Equal(t, MustToVolumePlan(literal), plan) - assert.Equal(t, plan.ToLiteral(), literal) - - assert.True(t, plan.Compatible(VolumePlan{ - MustToVolumeBinding("AUTO:/data0:ro:200"): VolumeMap{"/dir0": 200}, - MustToVolumeBinding("AUTO:/data1:rw:100"): VolumeMap{"/dir1": 100}, - })) - assert.False(t, plan.Compatible(VolumePlan{ - MustToVolumeBinding("AUTO:/data0:ro:200"): VolumeMap{"/dir0": 200}, - MustToVolumeBinding("AUTO:/data1:rw:100"): VolumeMap{"/dir2": 100}, - })) - - p := MustToVolumePlan(map[string]map[string]int64{ - "AUTO:/data1:w:100": { - "/tmp1": 100, - }, - "AUTO:/data2:w:100": { - "/tmp2": 100, - }, - }) - _, _, found := p.FindAffinityPlan(MustToVolumeBinding("AUTO:/data2:w")) - assert.True(t, found) - _, _, found = p.FindAffinityPlan(MustToVolumeBinding("AUTO:/data1:w")) - assert.True(t, found) - _, _, found = p.FindAffinityPlan(MustToVolumeBinding("AUTO:/data1:rw")) - assert.False(t, found) - _, _, found = p.FindAffinityPlan(MustToVolumeBinding("AUTO:/data3:w")) - assert.False(t, found) + assert.Equal(t, r.Int64("int64"), int64(1)) + assert.Equal(t, r.Int64("str-int"), int64(1)) + assert.Equal(t, r.Int64("float-int"), int64(2)) + assert.Equal(t, r.Float64("float64"), 1.999999999999999999999) + assert.Equal(t, r.String("string"), "string") + assert.Equal(t, r.StringSlice("string-slice"), []string{"string", "string"}) + assert.Equal(t, r.OneOfStringSlice("?", "string-slice"), []string{"string", "string"}) + assert.Equal(t, r.Bool("bool"), true) + assert.Equal(t, r.RawParams("raw-params")["int64"], 1) + assert.Equal(t, r.IsSet("?"), false) } diff --git a/types/tests.go b/types/tests.go deleted file mode 100644 index 234eb1b0c..000000000 --- a/types/tests.go +++ /dev/null @@ -1,34 +0,0 @@ -package types - -import "fmt" - -// MustToVolumeBinding convert volume string into VolumeBinding or panic -func MustToVolumeBinding(volume string) VolumeBinding { - vb, err := NewVolumeBinding(volume) - if err != nil { - panic(fmt.Errorf("invalid volume: %s", volume)) - } - return *vb -} - -// MustToVolumeBindings convert slice of volume string into VolumeBindings or panic -func MustToVolumeBindings(volumes []string) VolumeBindings { - vbs, err := NewVolumeBindings(volumes) - if err != nil { - panic(fmt.Errorf("invalid volumes: %s", volumes)) - } - return vbs -} - -// MustToVolumePlan convert VolumePlan from literal value -func MustToVolumePlan(plan map[string]map[string]int64) VolumePlan { - volumePlan := VolumePlan{} - for volume, volumeMap := range plan { - vb, err := NewVolumeBinding(volume) - if err != nil { - panic(fmt.Errorf("invalid plan %v: %v", plan, err)) - } - volumePlan[*vb] = VolumeMap(volumeMap) - } - return volumePlan -} diff --git a/types/volume_test.go b/types/volume_test.go deleted file mode 100644 index 0bdbf09ce..000000000 --- a/types/volume_test.go +++ /dev/null @@ -1,120 +0,0 @@ -package types - -import ( - "encoding/json" - "testing" - - "github.com/stretchr/testify/assert" -) - -func NormalVolumeBindingTestcases(t *testing.T) (testcases []*VolumeBinding) { - vb, err := NewVolumeBinding("/src:/dst:rwm:1000") - assert.Nil(t, err) - assert.Equal(t, vb, &VolumeBinding{"/src", "/dst", "mrw", int64(1000)}) - assert.False(t, vb.RequireSchedule()) - assert.False(t, vb.RequireScheduleMonopoly()) - testcases = append(testcases, vb) - - vb, err = NewVolumeBinding("/src:/dst:rwm") - assert.Nil(t, err) - assert.Equal(t, vb, &VolumeBinding{"/src", "/dst", "mrw", int64(0)}) - assert.False(t, vb.RequireSchedule()) - assert.False(t, vb.RequireScheduleMonopoly()) - testcases = append(testcases, vb) - - vb, err = NewVolumeBinding("/src:/dst") - assert.Nil(t, err) - assert.Equal(t, vb, &VolumeBinding{"/src", "/dst", "", int64(0)}) - assert.False(t, vb.RequireSchedule()) - assert.False(t, vb.RequireScheduleMonopoly()) - testcases = append(testcases, vb) - - return -} - -func AutoVolumeBindingTestcases(t *testing.T) (testcases []*VolumeBinding) { - vb, err := NewVolumeBinding("AUTO:/data:rw:1") - assert.Nil(t, err) - assert.True(t, vb.RequireSchedule()) - assert.False(t, vb.RequireScheduleMonopoly()) - testcases = append(testcases, vb) - - vb, err = NewVolumeBinding("AUTO:/dir:rwm:1") - assert.Nil(t, err) - assert.True(t, vb.RequireSchedule()) - assert.True(t, vb.RequireScheduleMonopoly()) - testcases = append(testcases, vb) - - return -} - -func TestNewVolumeBinding(t *testing.T) { - NormalVolumeBindingTestcases(t) - AutoVolumeBindingTestcases(t) - - _, err := NewVolumeBinding("/src:/dst:rw:1G") - assert.Error(t, err, "invalid syntax") - - _, err = NewVolumeBinding("/src:/dst:rwm:1:asdf") - assert.Error(t, err, "invalid volume") - - _, err = NewVolumeBinding("/src:/data:rw:-1") - assert.Nil(t, err) - - _, err = NewVolumeBinding("AUTO:/data:rw") - assert.Nil(t, err) - - _, err = NewVolumeBinding("AUTO::rw:1") - assert.Error(t, err, "dest must be provided") - - _, err = NewVolumeBinding("AUTO:/data:rmo:0") - assert.Error(t, err, "monopoly volume must not be limited") -} - -func TestVolumeBindingToString(t *testing.T) { - cases := NormalVolumeBindingTestcases(t) - assert.Equal(t, cases[0].ToString(false), "/src:/dst:mrw:1000") - assert.Equal(t, cases[1].ToString(false), "/src:/dst:mrw:0") - assert.Equal(t, cases[2].ToString(false), "/src:/dst") - assert.Equal(t, cases[1].ToString(true), "/src:/dst:rw:0") -} - -func TestVolumeBindings(t *testing.T) { - _, err := NewVolumeBindings([]string{"/1::rw:0"}) - assert.Error(t, err, "dest must be provided") - vbs, _ := NewVolumeBindings([]string{"/1:/dst:rw:1000", "/0:/dst:rom"}) - assert.Equal(t, vbs.ToStringSlice(false, false), []string{"/1:/dst:rw:1000", "/0:/dst:mro:0"}) - assert.Equal(t, vbs.ToStringSlice(true, false), []string{"/0:/dst:mro:0", "/1:/dst:rw:1000"}) - assert.Equal(t, vbs.TotalSize(), int64(1000)) - - vbs1, _ := NewVolumeBindings([]string{"AUTO:/data0:rw:1", "AUTO:/data1:rw:2", "/mnt1:/data2:rw", "/mnt2:/data3:ro"}) - vbs2, _ := NewVolumeBindings([]string{"AUTO:/data7:rw:3", "AUTO:/data1:rw:3", "/mnt3:/data8", "AUTO:/data0:rw:-20"}) - vbs = MergeVolumeBindings(vbs1, vbs2) - softVolumes, hardVolumes := vbs.Divide() - assert.Equal(t, softVolumes.ToStringSlice(true, false), []string{"AUTO:/data1:rw:5", "AUTO:/data7:rw:3"}) - assert.Equal(t, hardVolumes.ToStringSlice(true, false), []string{"/mnt1:/data2:rw:0", "/mnt2:/data3:ro:0", "/mnt3:/data8"}) - - assert.True(t, vbs1.IsEqual(vbs1)) - assert.False(t, vbs1.IsEqual(vbs2)) - - vp := VolumePlan{ - MustToVolumeBinding("AUTO:/data0:rw:1"): VolumeMap{"/mnt0": 1}, - MustToVolumeBinding("AUTO:/data1:rm:2"): VolumeMap{"/mnt1": 2}, - MustToVolumeBinding("AUTO:/data7:rw:3"): VolumeMap{"/mnt2": 3}, - } - vbs = vbs1.ApplyPlan(vp) - assert.True(t, MustToVolumeBindings([]string{"/mnt0:/data0:rw:1", "/mnt1:/data1:rw:2", "/mnt1:/data2:rw", "/mnt2:/data3:ro"}).IsEqual(vbs)) -} - -func TestVolumeBindingsJSONEncoding(t *testing.T) { - vbs := MustToVolumeBindings([]string{"AUTO:/data0:rw:1", "AUTO:/data1:rw:2", "/mnt1:/data2:rw", "/mnt2:/data3:ro"}) - data := []byte(`["AUTO:/data0:rw:1","AUTO:/data1:rw:2","/mnt1:/data2:rw:0","/mnt2:/data3:ro:0"]`) - b, err := json.Marshal(vbs) - assert.Nil(t, err) - assert.Equal(t, b, data) - - vbs1 := VolumeBindings{} - err = json.Unmarshal(data, &vbs1) - assert.Nil(t, err) - assert.Equal(t, vbs1, vbs) -} diff --git a/types/workload.go b/types/workload.go index 5302baea9..d4f796e2b 100644 --- a/types/workload.go +++ b/types/workload.go @@ -35,20 +35,21 @@ type LabelMeta struct { // only relationship with pod and node is stored // if you wanna get realtime information, use Inspect method type Workload struct { - 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"` - User string `json:"user"` - Env []string `json:"env"` - Image string `json:"image"` - Labels map[string]string `json:"labels"` - CreateTime int64 `json:"create_time"` - StatusMeta *StatusMeta `json:"-"` - Engine engine.API `json:"-"` + ResourceArgs ResourceMeta `json:"resource_args"` + EngineArgs EngineArgs `json:"engine_args"` + ID string `json:"id"` + Name string `json:"name"` + Podname string `json:"podname"` + Nodename string `json:"nodename"` + Hook *Hook `json:"hook"` + Privileged bool `json:"privileged"` + User string `json:"user"` + Env []string `json:"env"` + Image string `json:"image"` + Labels map[string]string `json:"labels"` + CreateTime int64 `json:"create_time"` + StatusMeta *StatusMeta `json:"-"` + Engine engine.API `json:"-"` } // Inspect a workload diff --git a/utils/file.go b/utils/file.go new file mode 100644 index 000000000..780d7dc82 --- /dev/null +++ b/utils/file.go @@ -0,0 +1,34 @@ +package utils + +import ( + "io/fs" + "path/filepath" +) + +const executablePerm = 0111 + +// ListAllExecutableFiles returns all the executable files in the given path +func ListAllExecutableFiles(basedir string) ([]string, error) { + files := []string{} + err := filepath.Walk(basedir, func(path string, info fs.FileInfo, err error) error { + if err != nil { + return err + } + if info.IsDir() && path != basedir { + return filepath.SkipDir + } + if !info.IsDir() && isExecutable(info.Mode().Perm()) { + files = append(files, path) + } + return nil + }) + + if err != nil { + return nil, err + } + return files, nil +} + +func isExecutable(perm fs.FileMode) bool { + return perm&executablePerm == executablePerm +} diff --git a/utils/file_test.go b/utils/file_test.go new file mode 100644 index 000000000..d4b585bf7 --- /dev/null +++ b/utils/file_test.go @@ -0,0 +1 @@ +package utils diff --git a/utils/generics.go b/utils/generics.go new file mode 100644 index 000000000..fcdbeb650 --- /dev/null +++ b/utils/generics.go @@ -0,0 +1,23 @@ +package utils + +// Map like map in Python +func Map[T1, T2 any](slice []T1, f func(T1) T2) []T2 { + result := make([]T2, len(slice)) + for i, v := range slice { + result[i] = f(v) + } + return result +} + +type addable interface { + ~int | ~int32 | ~int64 | ~uint | ~uint32 | ~uint64 | ~float32 | ~float64 | ~complex64 | ~complex128 +} + +// Sum returns sum of all elements in slice +func Sum[T addable](slice []T) T { + var result T + for _, v := range slice { + result += v + } + return result +} diff --git a/utils/generics_test.go b/utils/generics_test.go new file mode 100644 index 000000000..0d6a0d117 --- /dev/null +++ b/utils/generics_test.go @@ -0,0 +1,22 @@ +package utils + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestMap(t *testing.T) { + s1 := []int{1, 2, 3, 4, 5} + s2 := Map(s1, func(e int) string { + return fmt.Sprintf("%d", e) + }) + assert.Equal(t, []string{"1", "2", "3", "4", "5"}, s2) +} + +func TestSum(t *testing.T) { + s1 := []int{1, 2, 3, 4, 5} + s2 := Sum(s1) + assert.Equal(t, 15, s2) +} diff --git a/utils/ram.go b/utils/ram.go new file mode 100644 index 000000000..e2289e52c --- /dev/null +++ b/utils/ram.go @@ -0,0 +1,31 @@ +package utils + +import ( + "strconv" + "strings" + + "github.com/docker/go-units" +) + +// ParseRAMInHuman returns int value in bytes of a human readable string +// e.g. 100KB -> 102400 +func ParseRAMInHuman(ram string) (int64, error) { + if ram == "" { + return 0, nil + } + ramInBytes, err := strconv.ParseInt(ram, 10, 64) + if err == nil { + return ramInBytes, nil + } + + flag := int64(1) + if strings.HasPrefix(ram, "-") { + flag = int64(-1) + ram = strings.TrimLeft(ram, "-") + } + ramInBytes, err = units.RAMInBytes(ram) + if err != nil { + return 0, err + } + return ramInBytes * flag, nil +} diff --git a/utils/ram_test.go b/utils/ram_test.go new file mode 100644 index 000000000..84b66cc29 --- /dev/null +++ b/utils/ram_test.go @@ -0,0 +1,33 @@ +package utils + +import ( + "testing" + + "github.com/docker/go-units" + "github.com/stretchr/testify/assert" +) + +func TestParseRAMInHuman(t *testing.T) { + size, err := ParseRAMInHuman("") + assert.Nil(t, err) + assert.EqualValues(t, 0, size) + + size, err = ParseRAMInHuman("1") + assert.Nil(t, err) + assert.EqualValues(t, 1, size) + + size, err = ParseRAMInHuman("-1") + assert.Nil(t, err) + assert.EqualValues(t, -1, size) + + size, err = ParseRAMInHuman("hhhh") + assert.NotNil(t, err) + + size, err = ParseRAMInHuman("1G") + assert.Nil(t, err) + assert.EqualValues(t, units.GiB, size) + + size, err = ParseRAMInHuman("-1T") + assert.Nil(t, err) + assert.EqualValues(t, -units.TiB, size) +} diff --git a/utils/tests.go b/utils/tests.go deleted file mode 100644 index a632c6c15..000000000 --- a/utils/tests.go +++ /dev/null @@ -1,34 +0,0 @@ -package utils - -import ( - "fmt" - - resourcetypes "github.com/projecteru2/core/resources/types" - "github.com/projecteru2/core/types" -) - -// GenerateScheduleInfos generate nodes -func GenerateScheduleInfos(nums, cores int, memory, storage int64, shares int) []resourcetypes.ScheduleInfo { - var name string - scheduleInfos := []resourcetypes.ScheduleInfo{} - - for i := 0; i < nums; i++ { - name = fmt.Sprintf("n%d", i) - - cpumap := types.CPUMap{} - for j := 0; j < cores; j++ { - coreName := fmt.Sprintf("%d", j) - cpumap[coreName] = int64(shares) - } - scheduleInfo := resourcetypes.ScheduleInfo{ - NodeMeta: types.NodeMeta{ - CPU: cpumap, - MemCap: memory, - StorageCap: storage, - Name: name, - }, - } - scheduleInfos = append(scheduleInfos, scheduleInfo) - } - return scheduleInfos -} diff --git a/utils/tests_test.go b/utils/tests_test.go deleted file mode 100644 index 07be15ded..000000000 --- a/utils/tests_test.go +++ /dev/null @@ -1,12 +0,0 @@ -package utils - -import ( - "testing" - - "github.com/stretchr/testify/assert" -) - -func TestGenerateNodes(t *testing.T) { - ns := GenerateScheduleInfos(1, 10, 1000, 1000, 100) - assert.Len(t, ns, 1) -} diff --git a/utils/transaction.go b/utils/transaction.go index faf4c0059..60a1db219 100644 --- a/utils/transaction.go +++ b/utils/transaction.go @@ -30,6 +30,7 @@ func Txn(ctx context.Context, cond contextFunc, then contextFunc, rollback func( } log.Warnf(ctx, "[txn] txn failed, rolling back: %v", txnErr) + // forbid interrupting rollback rollbackCtx, rollBackCancel := context.WithTimeout(InheritTracingInfo(ctx, context.TODO()), ttl) defer rollBackCancel() @@ -53,3 +54,16 @@ func Txn(ctx context.Context, cond contextFunc, then contextFunc, rollback func( return txnErr } + +// PCR Prepare, Commit, Rollback. +// `prepare` should be a pure calculation process without side effects. +// `commit` writes the calculation result of `prepare` into database. +// if `commit` returns error, `rollback` will be performed. +func PCR(ctx context.Context, prepare func(ctx context.Context) error, commit func(ctx context.Context) error, rollback func(ctx context.Context) error, ttl time.Duration) error { + return Txn(ctx, prepare, commit, func(ctx context.Context, failureByCond bool) error { + if !failureByCond { + return rollback(ctx) + } + return nil + }, ttl) +}