Merge pull request #3789 from william-wang/colocation
Support colocation for computing workload and mircro-service
This commit is contained in:
commit
6e8dd2f1e3
|
|
@ -37,5 +37,5 @@ jobs:
|
|||
make verify
|
||||
make TAG=latest generate-yaml
|
||||
make verify-generated-yaml
|
||||
make unit-test
|
||||
sudo make unit-test
|
||||
working-directory: ./src/github.com/${{ github.repository }}
|
||||
|
|
|
|||
12
Makefile
12
Makefile
|
|
@ -65,7 +65,7 @@ include Makefile.def
|
|||
|
||||
.EXPORT_ALL_VARIABLES:
|
||||
|
||||
all: vc-scheduler vc-controller-manager vc-webhook-manager vcctl command-lines
|
||||
all: vc-scheduler vc-controller-manager vc-webhook-manager vc-agent vcctl command-lines
|
||||
|
||||
init:
|
||||
mkdir -p ${BIN_DIR}
|
||||
|
|
@ -84,14 +84,18 @@ vc-controller-manager: init
|
|||
vc-webhook-manager: init
|
||||
CC=${CC} CGO_ENABLED=0 go build -ldflags ${LD_FLAGS} -o ${BIN_DIR}/vc-webhook-manager ./cmd/webhook-manager
|
||||
|
||||
vc-agent: init
|
||||
CC=${CC} CGO_ENABLED=0 go build -ldflags ${LD_FLAGS} -o ${BIN_DIR}/vc-agent ./cmd/agent
|
||||
CC=${CC} CGO_ENABLED=0 go build -ldflags ${LD_FLAGS} -o ${BIN_DIR}/network-qos ./cmd/network-qos
|
||||
|
||||
vcctl: init
|
||||
CC=${CC} CGO_ENABLED=0 GOOS=${OS} go build -ldflags ${LD_FLAGS} -o ${BIN_DIR}/vcctl ./cmd/cli
|
||||
|
||||
image_bins: vc-scheduler vc-controller-manager vc-webhook-manager
|
||||
image_bins: vc-scheduler vc-controller-manager vc-webhook-manager vc-agent
|
||||
|
||||
images:
|
||||
for name in controller-manager scheduler webhook-manager; do\
|
||||
docker buildx build -t "${IMAGE_PREFIX}/vc-$$name:$(TAG)" . -f ./installer/dockerfile/$$name/Dockerfile --output=type=${BUILDX_OUTPUT_TYPE} --platform ${DOCKER_PLATFORMS} --build-arg APK_MIRROR=${APK_MIRROR}; \
|
||||
for name in controller-manager scheduler webhook-manager agent; do\
|
||||
docker buildx build -t "${IMAGE_PREFIX}/vc-$$name:$(TAG)" . -f ./installer/dockerfile/$$name/Dockerfile --output=type=${BUILDX_OUTPUT_TYPE} --platform ${DOCKER_PLATFORMS} --build-arg APK_MIRROR=${APK_MIRROR} --build-arg OPEN_EULER_IMAGE_TAG=${OPEN_EULER_IMAGE_TAG}; \
|
||||
done
|
||||
|
||||
generate-code:
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@ TAG ?= $(shell git rev-parse --verify HEAD)
|
|||
GitSHA=`git rev-parse HEAD`
|
||||
Date=`date "+%Y-%m-%d %H:%M:%S"`
|
||||
RELEASE_VER=latest
|
||||
OPEN_EULER_IMAGE_TAG ?= 22.03-lts-sp2
|
||||
LD_FLAGS=" \
|
||||
-X '${REPO_PATH}/pkg/version.GitSHA=${GitSHA}' \
|
||||
-X '${REPO_PATH}/pkg/version.Built=${Date}' \
|
||||
|
|
|
|||
|
|
@ -0,0 +1,97 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package app
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
"strings"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
cliflag "k8s.io/component-base/cli/flag"
|
||||
"k8s.io/component-base/cli/globalflag"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/cmd/agent/app/options"
|
||||
"volcano.sh/volcano/pkg/agent/events"
|
||||
"volcano.sh/volcano/pkg/agent/healthcheck"
|
||||
"volcano.sh/volcano/pkg/agent/utils"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
"volcano.sh/volcano/pkg/networkqos"
|
||||
|
||||
_ "volcano.sh/volcano/pkg/agent/oversubscription/policy/extend"
|
||||
)
|
||||
|
||||
func NewVolcanoAgentCommand(ctx context.Context) *cobra.Command {
|
||||
opts := options.NewVolcanoAgentOptions()
|
||||
cmd := &cobra.Command{
|
||||
Use: "volcano-agent",
|
||||
Long: `volcano-agent.`,
|
||||
Run: func(cmd *cobra.Command, args []string) {
|
||||
cliflag.PrintFlags(cmd.Flags())
|
||||
if err := Run(ctx, opts); err != nil {
|
||||
_, _ = fmt.Fprintf(os.Stderr, "%v\n", err)
|
||||
os.Exit(1)
|
||||
}
|
||||
},
|
||||
}
|
||||
|
||||
opts.AddFlags(cmd)
|
||||
globalflag.AddGlobalFlags(cmd.Flags(), cmd.Name())
|
||||
return cmd
|
||||
}
|
||||
|
||||
func Run(ctx context.Context, opts *options.VolcanoAgentOptions) error {
|
||||
klog.InfoS("Start volcano agent")
|
||||
conf, err := NewConfiguration(opts)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create volcano-agent configuration: %v", err)
|
||||
}
|
||||
|
||||
// TODO: get cgroup driver dynamically
|
||||
sfsFsPath := strings.TrimSpace(os.Getenv(utils.SysFsPathEnv))
|
||||
if sfsFsPath == "" {
|
||||
sfsFsPath = utils.DefaultSysFsPath
|
||||
}
|
||||
cgroupManager := cgroup.NewCgroupManager("cgroupfs", path.Join(sfsFsPath, "cgroup"), conf.GenericConfiguration.KubeCgroupRoot)
|
||||
metricCollectorManager, err := metriccollect.NewMetricCollectorManager(conf, cgroupManager)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create metric collector manager: %v", err)
|
||||
}
|
||||
|
||||
networkQoSMgr := networkqos.NewNetworkQoSManager(conf)
|
||||
err = networkQoSMgr.Init()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to init network qos: %v", err)
|
||||
}
|
||||
|
||||
eventManager := events.NewEventManager(conf, metricCollectorManager, cgroupManager)
|
||||
err = eventManager.Run(ctx)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to run event manager: %v", err)
|
||||
}
|
||||
|
||||
conf.InformerFactory.K8SInformerFactory.Start(ctx.Done())
|
||||
RunServer(healthcheck.NewHealthChecker(networkQoSMgr), conf.GenericConfiguration.HealthzAddress, conf.GenericConfiguration.HealthzPort)
|
||||
klog.InfoS("Volcano volcano-agent started")
|
||||
<-ctx.Done()
|
||||
klog.InfoS("Volcano volcano-agent stopped")
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package app
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net"
|
||||
"net/http"
|
||||
"strconv"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus/promhttp"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
clientset "k8s.io/client-go/kubernetes"
|
||||
"k8s.io/client-go/kubernetes/scheme"
|
||||
typedv1 "k8s.io/client-go/kubernetes/typed/core/v1"
|
||||
restclient "k8s.io/client-go/rest"
|
||||
"k8s.io/client-go/tools/record"
|
||||
"k8s.io/controller-manager/pkg/clientbuilder"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/cmd/agent/app/options"
|
||||
"volcano.sh/volcano/pkg/agent/healthcheck"
|
||||
"volcano.sh/volcano/pkg/agent/utils"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
)
|
||||
|
||||
func NewConfiguration(opts *options.VolcanoAgentOptions) (*config.Configuration, error) {
|
||||
conf := config.NewConfiguration()
|
||||
if err := opts.Validate(); err != nil {
|
||||
return conf, err
|
||||
}
|
||||
if err := opts.ApplyTo(conf); err != nil {
|
||||
return conf, err
|
||||
}
|
||||
|
||||
kubeConfig, err := restclient.InClusterConfig()
|
||||
if err != nil {
|
||||
return conf, fmt.Errorf("failed to create kubeconfig: %v", err)
|
||||
}
|
||||
|
||||
kubeClient, err := clientset.NewForConfig(restclient.AddUserAgent(kubeConfig, utils.Component))
|
||||
if err != nil {
|
||||
return conf, fmt.Errorf("failed to create kubeclient: %v", err)
|
||||
}
|
||||
conf.GenericConfiguration.KubeClient = kubeClient
|
||||
|
||||
broadcaster := record.NewBroadcaster()
|
||||
broadcaster.StartLogging(klog.Infof)
|
||||
broadcaster.StartStructuredLogging(2)
|
||||
broadcaster.StartRecordingToSink(&typedv1.EventSinkImpl{Interface: kubeClient.CoreV1().Events("")})
|
||||
recorder := broadcaster.NewRecorder(scheme.Scheme, corev1.EventSource{Component: "volcano-agent"})
|
||||
conf.GenericConfiguration.Recorder = recorder
|
||||
|
||||
rootClientBuilder := clientbuilder.SimpleControllerClientBuilder{
|
||||
ClientConfig: kubeConfig,
|
||||
}
|
||||
versionedClient := rootClientBuilder.ClientOrDie("shared-informers")
|
||||
conf.Complete(versionedClient)
|
||||
return conf, nil
|
||||
}
|
||||
|
||||
// RunServer will run both health check and metrics server.
|
||||
func RunServer(checker healthcheck.HealthChecker, address string, port int) {
|
||||
go func() {
|
||||
klog.InfoS("Start http health check server")
|
||||
mux := http.NewServeMux()
|
||||
mux.HandleFunc("/healthz", checker.HealthCheck)
|
||||
mux.Handle("/metrics", promhttp.Handler())
|
||||
s := &http.Server{
|
||||
Addr: net.JoinHostPort(address, strconv.Itoa(port)),
|
||||
Handler: mux,
|
||||
}
|
||||
if err := s.ListenAndServe(); err != nil {
|
||||
klog.Fatalf("failed to start health check server: %v", err)
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
|
@ -0,0 +1,105 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package options
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
)
|
||||
|
||||
const (
|
||||
serverPort = 3300
|
||||
|
||||
defaultOverSubscriptionRatio = 60
|
||||
)
|
||||
|
||||
type VolcanoAgentOptions struct {
|
||||
// HealthzAddress is the health check server address
|
||||
HealthzAddress string
|
||||
|
||||
//HealthzPort is the health check server port
|
||||
HealthzPort int
|
||||
|
||||
// KubeCgroupRoot is the root cgroup to use for pods.
|
||||
// If CgroupsPerQOS is enabled, this is the root of the QoS cgroup hierarchy.
|
||||
KubeCgroupRoot string
|
||||
|
||||
// KubeNodeName is the name of the node which controller is running.
|
||||
KubeNodeName string
|
||||
|
||||
// List of supported features, '*' supports all on-by-default features.
|
||||
SupportedFeatures []string
|
||||
|
||||
// KubePodName is the name of the pod.
|
||||
KubePodName string
|
||||
|
||||
// KubePodName is the namespace of the pod.
|
||||
KubePodNamespace string
|
||||
|
||||
// OverSubscriptionPolicy defines overSubscription policy.
|
||||
OverSubscriptionPolicy string
|
||||
|
||||
// OverSubscriptionRatio is the over subscription ratio of idle resources, default to 60, which means 60%.
|
||||
OverSubscriptionRatio int
|
||||
|
||||
// IncludeSystemUsage determines whether considering system usage when calculate overSubscription resource and evict.
|
||||
IncludeSystemUsage bool
|
||||
}
|
||||
|
||||
func NewVolcanoAgentOptions() *VolcanoAgentOptions {
|
||||
return &VolcanoAgentOptions{}
|
||||
}
|
||||
|
||||
func (options *VolcanoAgentOptions) AddFlags(c *cobra.Command) {
|
||||
c.Flags().StringSliceVar(&options.SupportedFeatures, "supported-features", []string{"*"}, "List of supported features. '*' supports all on-by-default features, 'foo' feature named 'foo' is supported"+
|
||||
"'-foo' feature named 'foo' is not supported.")
|
||||
c.Flags().StringVar(&options.HealthzAddress, "healthz-address", "", "defines the health check address")
|
||||
c.Flags().IntVar(&options.HealthzPort, "healthz-port", serverPort, "defines the health check port")
|
||||
c.Flags().StringVar(&options.KubeCgroupRoot, "kube-cgroup-root", options.KubeCgroupRoot, "kube cgroup root")
|
||||
c.Flags().StringVar(&options.KubeNodeName, "kube-node-name", os.Getenv("KUBE_NODE_NAME"), "the related kube-node name of the host, where the pod run in")
|
||||
c.Flags().StringVar(&options.KubePodName, "kube-pod-name", os.Getenv("KUBE_POD_NAME"), "the name of the pod")
|
||||
c.Flags().StringVar(&options.KubePodNamespace, "kube-pod-namespace", os.Getenv("KUBE_POD_NAMESPACE"), "the namespace of the pod")
|
||||
c.Flags().StringVar(&options.OverSubscriptionPolicy, "oversubscription-policy", "extend", "The oversubscription policy determines where oversubscription resources to report and how to use, default to extend means report to extend resources")
|
||||
// TODO: put in configMap.
|
||||
c.Flags().IntVar(&options.OverSubscriptionRatio, "oversubscription-ratio", defaultOverSubscriptionRatio, "The oversubscription ratio determines how many idle resources can be oversold")
|
||||
c.Flags().BoolVar(&options.IncludeSystemUsage, "include-system-usage", false, "It determines whether considering system usage when calculate overSubscription resource and evict.")
|
||||
}
|
||||
|
||||
func (options *VolcanoAgentOptions) Validate() error {
|
||||
if options.OverSubscriptionRatio <= 0 {
|
||||
return fmt.Errorf("over subscription ratio must be greater than 0")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (options *VolcanoAgentOptions) ApplyTo(cfg *config.Configuration) error {
|
||||
cfg.GenericConfiguration.HealthzAddress = options.HealthzAddress
|
||||
cfg.GenericConfiguration.HealthzPort = options.HealthzPort
|
||||
cfg.GenericConfiguration.KubeCgroupRoot = options.KubeCgroupRoot
|
||||
cfg.GenericConfiguration.KubeNodeName = options.KubeNodeName
|
||||
cfg.GenericConfiguration.SupportedFeatures = options.SupportedFeatures
|
||||
cfg.GenericConfiguration.KubePodName = options.KubePodName
|
||||
cfg.GenericConfiguration.KubePodNamespace = options.KubePodNamespace
|
||||
cfg.GenericConfiguration.OverSubscriptionPolicy = options.OverSubscriptionPolicy
|
||||
cfg.GenericConfiguration.OverSubscriptionRatio = options.OverSubscriptionRatio
|
||||
cfg.GenericConfiguration.IncludeSystemUsage = options.IncludeSystemUsage
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package options
|
||||
|
||||
import "testing"
|
||||
|
||||
func TestVolcanoAgentOptions_Validate(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
OverSubscriptionRatio int
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
name: "over subscription ratio lower than 0",
|
||||
OverSubscriptionRatio: -1,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "over subscription ratio greater than 100",
|
||||
OverSubscriptionRatio: 110,
|
||||
wantErr: false,
|
||||
},
|
||||
{
|
||||
name: "valid over subscription ratio",
|
||||
OverSubscriptionRatio: 80,
|
||||
wantErr: false,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
options := &VolcanoAgentOptions{
|
||||
OverSubscriptionRatio: tt.OverSubscriptionRatio,
|
||||
}
|
||||
if err := options.Validate(); (err != nil) != tt.wantErr {
|
||||
t.Errorf("Validate() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"k8s.io/component-base/logs"
|
||||
"sigs.k8s.io/controller-runtime/pkg/manager/signals"
|
||||
|
||||
"volcano.sh/volcano/cmd/agent/app"
|
||||
)
|
||||
|
||||
func main() {
|
||||
stopCtx := signals.SetupSignalHandler()
|
||||
command := app.NewVolcanoAgentCommand(stopCtx)
|
||||
|
||||
logs.InitLogs()
|
||||
defer logs.FlushLogs()
|
||||
|
||||
if err := command.Execute(); err != nil {
|
||||
_, _ = fmt.Fprintf(os.Stderr, "%v\n", err)
|
||||
os.Exit(1)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,179 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package cni
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
cilliumbpf "github.com/cilium/ebpf"
|
||||
"github.com/containernetworking/cni/pkg/skel"
|
||||
cnitypes "github.com/containernetworking/cni/pkg/types"
|
||||
cnitypesver "github.com/containernetworking/cni/pkg/types/100"
|
||||
cniversion "github.com/containernetworking/cni/pkg/version"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos/api"
|
||||
"volcano.sh/volcano/pkg/networkqos/tc"
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
func Execute() {
|
||||
err := utils.InitLog(utils.CNILogFilePath)
|
||||
if err != nil {
|
||||
fmt.Printf("failed fo init log: %v\n", err)
|
||||
}
|
||||
|
||||
err = PluginMain()
|
||||
if err != nil {
|
||||
klog.Flush()
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
klog.Flush()
|
||||
os.Exit(0)
|
||||
}
|
||||
|
||||
func PluginMain() error {
|
||||
e := skel.PluginMainWithError(
|
||||
func(args *skel.CmdArgs) error {
|
||||
addErr := cmdAdd(args)
|
||||
if addErr != nil {
|
||||
return fmt.Errorf("CNI add request failed, ContainerID(%s) Netns(%s) IfName(%s) err: %v", args.ContainerID, args.Netns, args.IfName, addErr)
|
||||
}
|
||||
return nil
|
||||
},
|
||||
func(args *skel.CmdArgs) error {
|
||||
return nil
|
||||
},
|
||||
func(args *skel.CmdArgs) error {
|
||||
delErr := cmdDel(args)
|
||||
if delErr != nil {
|
||||
return fmt.Errorf("CNI delete request failed, ContainerID(%s) Netns(%s) IfName(%s) err: %v", args.ContainerID, args.Netns, args.IfName, delErr)
|
||||
}
|
||||
return nil
|
||||
},
|
||||
cniversion.All, "CNI network-qos plugin")
|
||||
|
||||
if e != nil {
|
||||
klog.ErrorS(e, "Failed CNI request")
|
||||
if err := e.Print(); err != nil {
|
||||
klog.ErrorS(err, "Error writing error JSON to stdout: ", err)
|
||||
}
|
||||
return e
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func cmdAdd(args *skel.CmdArgs) (err error) {
|
||||
klog.InfoS("CNI add request received", "containerID", args.ContainerID,
|
||||
"netns", args.Netns, "ifName", args.IfName, "args", args.Args, "path", args.Path, "stdinData", args.StdinData)
|
||||
|
||||
k8sArgs := &api.K8sArgs{}
|
||||
err = cnitypes.LoadArgs(args.Args, k8sArgs)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to load k8s config from args: %v", err)
|
||||
}
|
||||
|
||||
confRequest := &api.NetConf{}
|
||||
err = json.Unmarshal(args.StdinData, confRequest)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to load net config from stdin data: %v", err)
|
||||
}
|
||||
|
||||
if err := cniversion.ParsePrevResult(&confRequest.NetConf); err != nil {
|
||||
return fmt.Errorf("could not parse prevResult: %v", err)
|
||||
}
|
||||
|
||||
if confRequest.PrevResult == nil {
|
||||
return fmt.Errorf("must be called as chained plugin")
|
||||
}
|
||||
|
||||
prevResult, err := cnitypesver.NewResultFromResult(confRequest.PrevResult)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to convert prevResult: %v", err)
|
||||
}
|
||||
result := prevResult
|
||||
|
||||
addErr := add(confRequest, args)
|
||||
if addErr != nil {
|
||||
return fmt.Errorf("failed to add cni: %v", addErr)
|
||||
}
|
||||
|
||||
err = cnitypes.PrintResult(result, confRequest.CNIVersion)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
klog.InfoS("CNI add request successfully", "containerID", args.ContainerID,
|
||||
"netns", args.Netns, "ifName", args.IfName, "args", args.Args, "path", args.Path, "stdinData", args.StdinData)
|
||||
return nil
|
||||
}
|
||||
|
||||
func add(confRequest *api.NetConf, args *skel.CmdArgs) error {
|
||||
_, err := throttling.GetNetworkThrottlingConfig().GetThrottlingConfig()
|
||||
if err != nil {
|
||||
// restart the node ebpf map will be lost
|
||||
if errors.Is(err, cilliumbpf.ErrKeyNotExist) || os.IsNotExist(err) {
|
||||
if confRequest.Args[utils.NodeColocationEnable] == "true" {
|
||||
_, err = throttling.GetNetworkThrottlingConfig().CreateThrottlingConfig(confRequest.Args[utils.OnlineBandwidthWatermarkKey], confRequest.Args[utils.OfflineLowBandwidthKey],
|
||||
confRequest.Args[utils.OfflineHighBandwidthKey], confRequest.Args[utils.NetWorkQoSCheckInterval])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
} else {
|
||||
return fmt.Errorf("failed to get throttling config: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
support, err := tc.GetTCCmd().PreAddFilter(args.Netns, args.IfName)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to set tc: %v", err)
|
||||
}
|
||||
if support {
|
||||
err = tc.GetTCCmd().AddFilter(args.Netns, args.IfName)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to set tc: %v", err)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func cmdDel(args *skel.CmdArgs) error {
|
||||
klog.InfoS("CNI delete request received", "containerID", args.ContainerID,
|
||||
"netns", args.Netns, "ifName", args.IfName, "args", args.Args, "path", args.Path, "stdinData", args.StdinData)
|
||||
|
||||
k8sArgs := &api.K8sArgs{}
|
||||
err := cnitypes.LoadArgs(args.Args, k8sArgs)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to load k8s config from args: %v", err)
|
||||
}
|
||||
|
||||
err = tc.GetTCCmd().RemoveFilter(args.Netns, args.IfName)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to delete tc: %v", err)
|
||||
}
|
||||
|
||||
klog.InfoS("CNI delete request successfully", "containerID", args.ContainerID,
|
||||
"netns", args.Netns, "ifName", args.IfName, "args", args.Args, "path", args.Path, "stdinData", args.StdinData)
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,221 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package cni
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
cilliumbpf "github.com/cilium/ebpf"
|
||||
"github.com/containernetworking/cni/pkg/skel"
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos/api"
|
||||
"volcano.sh/volcano/pkg/networkqos/tc"
|
||||
mocktc "volcano.sh/volcano/pkg/networkqos/tc/mocks"
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
mockthrottling "volcano.sh/volcano/pkg/networkqos/throttling/mocks"
|
||||
)
|
||||
|
||||
var fileWithoutNetworkQos = `{
|
||||
"cniVersion": "0.3.1",
|
||||
"name": "default-network",
|
||||
"args": {
|
||||
"check-interval": "10000000",
|
||||
"offline-high-bandwidth": "50MB",
|
||||
"offline-low-bandwidth": "20MB",
|
||||
"colocation": "true",
|
||||
"online-bandwidth-watermark": "50MB"
|
||||
},
|
||||
"name": "network-qos",
|
||||
"type": "network-qos",
|
||||
"prevResult": {
|
||||
"cniVersion":"0.3.1",
|
||||
"ips":[{"version":"4","address":"10.3.3.190/17"}],
|
||||
"dns":{}
|
||||
}
|
||||
}`
|
||||
|
||||
func TestCmdAdd(t *testing.T) {
|
||||
mockController := gomock.NewController(t)
|
||||
defer mockController.Finish()
|
||||
|
||||
mockTc := mocktc.NewMockTC(mockController)
|
||||
tc.SetTcCmd(mockTc)
|
||||
mockThr := mockthrottling.NewMockThrottlingConfig(mockController)
|
||||
throttling.SetNetworkThrottlingConfig(mockThr)
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
args *skel.CmdArgs
|
||||
apiCall []*gomock.Call
|
||||
expectedError bool
|
||||
}{
|
||||
{
|
||||
name: "add dev successful",
|
||||
args: &skel.CmdArgs{
|
||||
Netns: "test-ns",
|
||||
IfName: "eth0",
|
||||
StdinData: []byte(fileWithoutNetworkQos),
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 10000000,
|
||||
WaterLine: 1000,
|
||||
LowRate: 1000,
|
||||
HighRate: 1000,
|
||||
}, nil),
|
||||
mockTc.EXPECT().PreAddFilter("test-ns", "eth0").Return(true, nil),
|
||||
mockTc.EXPECT().AddFilter("test-ns", "eth0").Return(nil),
|
||||
},
|
||||
expectedError: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "throttling conf map not exists && throttling enabled",
|
||||
args: &skel.CmdArgs{
|
||||
Netns: "test-ns2",
|
||||
IfName: "eth0",
|
||||
StdinData: []byte(fileWithoutNetworkQos),
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(nil, cilliumbpf.ErrKeyNotExist),
|
||||
mockThr.EXPECT().CreateThrottlingConfig("50MB", "20MB", "50MB", "10000000").Return(&api.EbpfNetThrottlingConfig{
|
||||
WaterLine: 1000,
|
||||
LowRate: 1000,
|
||||
HighRate: 1000,
|
||||
Interval: 10000000,
|
||||
}, nil),
|
||||
mockTc.EXPECT().PreAddFilter("test-ns2", "eth0").Return(true, nil),
|
||||
mockTc.EXPECT().AddFilter("test-ns2", "eth0").Return(nil),
|
||||
},
|
||||
expectedError: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "throttling conf map pinned file not exists && throttling enabled",
|
||||
args: &skel.CmdArgs{
|
||||
Netns: "test-ns3",
|
||||
IfName: "eth0",
|
||||
StdinData: []byte(fileWithoutNetworkQos),
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(nil, os.ErrNotExist),
|
||||
mockThr.EXPECT().CreateThrottlingConfig("50MB", "20MB", "50MB", "10000000").Return(&api.EbpfNetThrottlingConfig{
|
||||
WaterLine: 1000,
|
||||
LowRate: 1000,
|
||||
HighRate: 1000,
|
||||
Interval: 10000000,
|
||||
}, nil),
|
||||
mockTc.EXPECT().PreAddFilter("test-ns3", "eth0").Return(true, nil),
|
||||
mockTc.EXPECT().AddFilter("test-ns3", "eth0").Return(nil),
|
||||
},
|
||||
expectedError: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "throttling conf map pinned file not exists && throttling enabled && create conf map failed",
|
||||
args: &skel.CmdArgs{
|
||||
Netns: "test-ns4",
|
||||
IfName: "eth0",
|
||||
StdinData: []byte(fileWithoutNetworkQos),
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(nil, os.ErrNotExist),
|
||||
mockThr.EXPECT().CreateThrottlingConfig("50MB", "20MB", "50MB", "10000000").Return(nil, fmt.Errorf("create failed")),
|
||||
},
|
||||
expectedError: true,
|
||||
},
|
||||
|
||||
{
|
||||
name: "add tc filter failed",
|
||||
args: &skel.CmdArgs{
|
||||
Netns: "test-ns5",
|
||||
IfName: "eth0",
|
||||
StdinData: []byte(fileWithoutNetworkQos),
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(nil, os.ErrNotExist),
|
||||
mockThr.EXPECT().CreateThrottlingConfig("50MB", "20MB", "50MB", "10000000").Return(&api.EbpfNetThrottlingConfig{
|
||||
WaterLine: 1000,
|
||||
LowRate: 1000,
|
||||
HighRate: 1000,
|
||||
Interval: 10000000,
|
||||
}, nil),
|
||||
mockTc.EXPECT().PreAddFilter("test-ns5", "eth0").Return(true, nil),
|
||||
mockTc.EXPECT().AddFilter("test-ns5", "eth0").Return(fmt.Errorf("add failed")),
|
||||
},
|
||||
expectedError: true,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
gomock.InOrder(tc.apiCall...)
|
||||
actualErr := cmdAdd(tc.args)
|
||||
assert.Equal(t, tc.expectedError, actualErr != nil, tc.name, actualErr)
|
||||
}
|
||||
}
|
||||
|
||||
func TestCmdDel(t *testing.T) {
|
||||
mockController := gomock.NewController(t)
|
||||
defer mockController.Finish()
|
||||
|
||||
mockTc := mocktc.NewMockTC(mockController)
|
||||
tc.SetTcCmd(mockTc)
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
args *skel.CmdArgs
|
||||
apiCall []*gomock.Call
|
||||
expectedError bool
|
||||
}{
|
||||
|
||||
{
|
||||
name: "delete dev successful",
|
||||
args: &skel.CmdArgs{
|
||||
Netns: "test-ns",
|
||||
IfName: "eth0",
|
||||
StdinData: []byte(fileWithoutNetworkQos),
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockTc.EXPECT().RemoveFilter("test-ns", "eth0").Return(nil),
|
||||
},
|
||||
expectedError: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "delete tc filter failed",
|
||||
args: &skel.CmdArgs{
|
||||
Netns: "test-ns2",
|
||||
IfName: "eth0",
|
||||
StdinData: []byte(fileWithoutNetworkQos),
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockTc.EXPECT().RemoveFilter("test-ns2", "eth0").Return(fmt.Errorf("delete failed")),
|
||||
},
|
||||
expectedError: true,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
gomock.InOrder(tc.apiCall...)
|
||||
actualErr := cmdDel(tc.args)
|
||||
assert.Equal(t, tc.expectedError, actualErr != nil, tc.name, actualErr)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package main
|
||||
|
||||
import (
|
||||
"os"
|
||||
|
||||
"volcano.sh/volcano/cmd/network-qos/cni"
|
||||
"volcano.sh/volcano/cmd/network-qos/tools"
|
||||
)
|
||||
|
||||
func main() {
|
||||
switch os.Getenv("CNI_COMMAND") {
|
||||
case "":
|
||||
tools.Execute()
|
||||
default:
|
||||
cni.Execute()
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,85 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"strconv"
|
||||
|
||||
cilliumbpf "github.com/cilium/ebpf"
|
||||
"github.com/spf13/cobra"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos/api"
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
type getCmd struct {
|
||||
out io.Writer
|
||||
errOut io.Writer
|
||||
}
|
||||
|
||||
func newGetCmd(out io.Writer, errOut io.Writer) *cobra.Command {
|
||||
get := &getCmd{
|
||||
out: out,
|
||||
errOut: errOut,
|
||||
}
|
||||
|
||||
return &cobra.Command{
|
||||
Use: "show",
|
||||
Short: "Show online-bandwidth-watermark/offline-low-bandwidth/offline-high-bandwidth config values in ebpf map",
|
||||
Long: `Show online-bandwidth-watermark/offline-low-bandwidth/offline-high-bandwidth config values in ebpf map`,
|
||||
Run: func(cmd *cobra.Command, args []string) {
|
||||
klog.InfoS("Network QoS command called", "command", "show")
|
||||
err := get.run()
|
||||
if err != nil {
|
||||
utils.Error(get.errOut, cmd, err)
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (c *getCmd) run() (err error) {
|
||||
config, err := throttling.GetNetworkThrottlingConfig().GetThrottlingConfig()
|
||||
if err != nil {
|
||||
if errors.Is(err, cilliumbpf.ErrKeyNotExist) || os.IsNotExist(err) {
|
||||
return fmt.Errorf("failed to get throttling config: %v, network qos has not been initialized, please enable network qos first", err)
|
||||
}
|
||||
return fmt.Errorf("failed to get throttling config: %v", err)
|
||||
}
|
||||
|
||||
configResult := &api.EbpfNetThrottlingConfGetResult{
|
||||
WaterLine: strconv.FormatUint(config.WaterLine*8, 10) + "bps",
|
||||
Interval: config.Interval,
|
||||
LowRate: strconv.FormatUint(config.LowRate*8, 10) + "bps",
|
||||
HighRate: strconv.FormatUint(config.HighRate*8, 10) + "bps",
|
||||
}
|
||||
|
||||
throttlingConfig, err := json.Marshal(configResult)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to marshal throttling config %v to json: %v", config, err)
|
||||
}
|
||||
|
||||
fmt.Fprintf(c.out, "%s: %s\n", "network-qos config", throttlingConfig)
|
||||
klog.InfoS("Network QoS command called successfully", "command", "show", "throttling-config", throttlingConfig)
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/agiledragon/gomonkey/v2"
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos/api"
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
mockthrottling "volcano.sh/volcano/pkg/networkqos/throttling/mocks"
|
||||
)
|
||||
|
||||
func TestGetCmdExecute(t *testing.T) {
|
||||
mockController := gomock.NewController(t)
|
||||
defer mockController.Finish()
|
||||
|
||||
mockThr := mockthrottling.NewMockThrottlingConfig(mockController)
|
||||
throttling.SetNetworkThrottlingConfig(mockThr)
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
apiCall []*gomock.Call
|
||||
jsonMarshalErr error
|
||||
expectedOut string
|
||||
expectedErrOut string
|
||||
expectedOsExitCalled bool
|
||||
}{
|
||||
{
|
||||
name: "[show] json Marshal failed",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 10000000,
|
||||
WaterLine: 2000,
|
||||
LowRate: 3000,
|
||||
HighRate: 4000,
|
||||
}, nil),
|
||||
},
|
||||
jsonMarshalErr: fmt.Errorf("json marshal failed"),
|
||||
expectedErrOut: `execute command[show] failed, error:failed to marshal throttling config &{2000 10000000 3000 4000} to json: json marshal failed` + "\n",
|
||||
expectedOsExitCalled: true,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[show] get conf successfully",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 10000000,
|
||||
WaterLine: 1500,
|
||||
LowRate: 2500,
|
||||
HighRate: 3500,
|
||||
}, nil),
|
||||
},
|
||||
expectedOut: `network-qos config: {"online_bandwidth_watermark":"12000bps","interval":10000000,"offline_low_bandwidth":"20000bps","offline_high_bandwidth":"28000bps"}` + "\n",
|
||||
},
|
||||
|
||||
{
|
||||
name: "[show] get conf failed",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(nil, fmt.Errorf("request failed")),
|
||||
},
|
||||
expectedErrOut: `execute command[show] failed, error:failed to get throttling config: request failed` + "\n",
|
||||
expectedOsExitCalled: true,
|
||||
},
|
||||
}
|
||||
|
||||
exitCalled := false
|
||||
getPatch := gomonkey.NewPatches()
|
||||
getPatch.ApplyFunc(os.Exit, func(code int) {
|
||||
exitCalled = true
|
||||
})
|
||||
defer getPatch.Reset()
|
||||
|
||||
for _, tc := range testCases {
|
||||
exitCalled = false
|
||||
var jsonMarshalPatch *gomonkey.Patches
|
||||
if tc.jsonMarshalErr != nil {
|
||||
jsonMarshalPatch = gomonkey.NewPatches()
|
||||
jsonMarshalPatch.ApplyFunc(json.Marshal, func(v interface{}) ([]byte, error) {
|
||||
return nil, tc.jsonMarshalErr
|
||||
})
|
||||
}
|
||||
out := &bytes.Buffer{}
|
||||
errOut := &bytes.Buffer{}
|
||||
cmd := newGetCmd(out, errOut)
|
||||
cmd.Execute()
|
||||
if jsonMarshalPatch != nil {
|
||||
jsonMarshalPatch.Reset()
|
||||
}
|
||||
assert.Equal(t, tc.expectedOsExitCalled, exitCalled, tc.name)
|
||||
assert.Equal(t, tc.expectedOut, out.String(), tc.name)
|
||||
assert.Equal(t, tc.expectedErrOut, errOut.String(), tc.name)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package options
|
||||
|
||||
import (
|
||||
"github.com/spf13/cobra"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
type Options struct {
|
||||
CheckoutInterval string
|
||||
OnlineBandwidthWatermark string
|
||||
OfflineLowBandwidth string
|
||||
OfflineHighBandwidth string
|
||||
EnableNetworkQoS bool
|
||||
}
|
||||
|
||||
// AddFlags is responsible for add flags from the given FlagSet instance for current GenericOptions.
|
||||
func (o *Options) AddFlags(c *cobra.Command) {
|
||||
c.Flags().StringVar(&o.CheckoutInterval, utils.NetWorkQoSCheckInterval, o.CheckoutInterval, "check interval is the interval of checking and updating the offline jobs bandwidth limit")
|
||||
c.Flags().StringVar(&o.OnlineBandwidthWatermark, utils.OnlineBandwidthWatermarkKey, o.OnlineBandwidthWatermark, "online-bandwidth-watermark is is the bandwidth threshold of online jobs, "+
|
||||
"is the sum of bandwidth of all online pods")
|
||||
c.Flags().StringVar(&o.OfflineLowBandwidth, utils.OfflineLowBandwidthKey, o.OfflineLowBandwidth, "offline-low-bandwidth is the maximum amount of network bandwidth that can be used by offline jobs when the"+
|
||||
"bandwidth usage of online jobs exceeds the defined threshold(online-bandwidth-watermark)")
|
||||
c.Flags().StringVar(&o.OfflineHighBandwidth, utils.OfflineHighBandwidthKey, o.OfflineHighBandwidth, "offline-high-bandwidth is the maximum amount of network bandwidth that can be used by offline jobs when the"+
|
||||
"bandwidth usage of online jobs not reach to the defined threshold(online-bandwidth-watermark)")
|
||||
c.Flags().BoolVar(&o.EnableNetworkQoS, utils.EnableNetworkQoS, o.EnableNetworkQoS, "enbale networkqos")
|
||||
}
|
||||
|
|
@ -0,0 +1,57 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package options
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestAddFlags(t *testing.T) {
|
||||
testCases := []struct {
|
||||
name string
|
||||
args []string
|
||||
opts *Options
|
||||
expectedOptions *Options
|
||||
expectedErr bool
|
||||
}{
|
||||
{
|
||||
name: "test set flags",
|
||||
args: []string{"--check-interval=20000", "--online-bandwidth-watermark=500Mbps", "--offline-low-bandwidth=300Mbps", "--offline-high-bandwidth=700Mbps"},
|
||||
opts: &Options{},
|
||||
expectedOptions: &Options{
|
||||
CheckoutInterval: "20000",
|
||||
OnlineBandwidthWatermark: "500Mbps",
|
||||
OfflineLowBandwidth: "300Mbps",
|
||||
OfflineHighBandwidth: "700Mbps",
|
||||
},
|
||||
expectedErr: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
cmds := &cobra.Command{}
|
||||
tc.opts.AddFlags(cmds)
|
||||
parseErr := cmds.Flags().Parse(tc.args)
|
||||
if parseErr != nil {
|
||||
t.Errorf("unexpected parse err: %v", parseErr)
|
||||
}
|
||||
assert.Equal(t, tc.expectedOptions, tc.opts, tc.name)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,133 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/utils/exec"
|
||||
"volcano.sh/volcano/pkg/networkqos/cni"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
type prepareCmd struct {
|
||||
out io.Writer
|
||||
errOut io.Writer
|
||||
}
|
||||
|
||||
func newPrepareCmd(out io.Writer, errOut io.Writer) *cobra.Command {
|
||||
prepare := &prepareCmd{
|
||||
out: out,
|
||||
errOut: errOut,
|
||||
}
|
||||
|
||||
cmd := &cobra.Command{
|
||||
Use: "prepare",
|
||||
Short: "prepare online-bandwidth-watermark/offline-low-bandwidth/offline-high-bandwidth values",
|
||||
Long: "prepare online-bandwidth-watermark/offline-low-bandwidth/offline-high-bandwidth values",
|
||||
Run: func(cmd *cobra.Command, args []string) {
|
||||
klog.InfoS("Network QoS command called", "command", "prepare")
|
||||
err := prepare.run()
|
||||
if err != nil {
|
||||
utils.Error(prepare.errOut, cmd, err)
|
||||
return
|
||||
}
|
||||
},
|
||||
}
|
||||
(&opt).AddFlags(cmd)
|
||||
return cmd
|
||||
}
|
||||
|
||||
func (c *prepareCmd) run() (err error) {
|
||||
enableNetworkQoS := opt.EnableNetworkQoS
|
||||
onlineBandwidthWatermark, offlineLowBandwidth, offlineHighBandwidth, interval := opt.OnlineBandwidthWatermark,
|
||||
opt.OfflineLowBandwidth, opt.OfflineHighBandwidth, opt.CheckoutInterval
|
||||
|
||||
if !enableNetworkQoS {
|
||||
return c.unInstall()
|
||||
}
|
||||
|
||||
return c.install(onlineBandwidthWatermark, offlineLowBandwidth, offlineHighBandwidth, interval)
|
||||
}
|
||||
|
||||
func (c *prepareCmd) unInstall() error {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer cancel()
|
||||
cmd := fmt.Sprintf(utils.NetWorkCmdFile + " reset")
|
||||
output, err := exec.GetExecutor().CommandContext(ctx, cmd)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to reset network qos:%v, output:%s", err, output)
|
||||
}
|
||||
|
||||
cniConfFile := strings.TrimSpace(os.Getenv(utils.CNIConfFilePathEnv))
|
||||
if cniConfFile == "" {
|
||||
cniConfFile = utils.DefaultCNIConfFile
|
||||
}
|
||||
err = cni.GetCNIPluginConfHandler().DeleteCniPluginFromConfList(cniConfFile, utils.CNIPluginName)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to reset cni config: %v", err)
|
||||
}
|
||||
klog.InfoS("Network QoS command called successfully", "command", "prepare[uninstall]")
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *prepareCmd) install(onlineBandwidthWatermark, offlineLowBandwidth, offlineHighBandwidth, interval string) error {
|
||||
if len(interval) == 0 {
|
||||
interval = utils.DefaultInterval
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer cancel()
|
||||
cmd := fmt.Sprintf(utils.NetWorkCmdFile+" set --%s=%s --%s=%s --%s=%s --%s=%s", utils.OnlineBandwidthWatermarkKey, onlineBandwidthWatermark,
|
||||
utils.OfflineLowBandwidthKey, offlineLowBandwidth,
|
||||
utils.OfflineHighBandwidthKey, offlineHighBandwidth,
|
||||
utils.NetWorkQoSCheckInterval, interval)
|
||||
output, err := exec.GetExecutor().CommandContext(ctx, cmd)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to set network qos:%v, output:%s", err, output)
|
||||
}
|
||||
|
||||
cniConf := make(map[string]interface{})
|
||||
cniConf["name"] = utils.CNIPluginName
|
||||
cniConf["type"] = utils.CNIPluginName
|
||||
cniConf["args"] = map[string]string{
|
||||
utils.NodeColocationEnable: "true",
|
||||
utils.NetWorkQoSCheckInterval: interval,
|
||||
utils.OnlineBandwidthWatermarkKey: onlineBandwidthWatermark,
|
||||
utils.OfflineLowBandwidthKey: offlineLowBandwidth,
|
||||
utils.OfflineHighBandwidthKey: offlineHighBandwidth,
|
||||
}
|
||||
|
||||
cniConfFile := strings.TrimSpace(os.Getenv(utils.CNIConfFilePathEnv))
|
||||
if cniConfFile == "" {
|
||||
cniConfFile = utils.DefaultCNIConfFile
|
||||
}
|
||||
err = cni.GetCNIPluginConfHandler().AddOrUpdateCniPluginToConfList(cniConfFile, utils.CNIPluginName, cniConf)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to add/update cni plugin to configlist: %v", err)
|
||||
}
|
||||
klog.InfoS("Network QoS command called successfully", "command", "prepare[install]", "cni-conf", cniConf)
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,198 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/utils/exec"
|
||||
mockexec "volcano.sh/volcano/pkg/agent/utils/exec/mocks"
|
||||
"volcano.sh/volcano/pkg/networkqos/cni"
|
||||
mockcni "volcano.sh/volcano/pkg/networkqos/cni/mocks"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
func TestUnInstall(t *testing.T) {
|
||||
mockController := gomock.NewController(t)
|
||||
defer mockController.Finish()
|
||||
|
||||
mockExec := mockexec.NewMockExecInterface(mockController)
|
||||
exec.SetExecutor(mockExec)
|
||||
mockCniConf := mockcni.NewMockCNIPluginsConfHandler(mockController)
|
||||
cni.SetCNIPluginConfHandler(mockCniConf)
|
||||
|
||||
out := &bytes.Buffer{}
|
||||
errOut := &bytes.Buffer{}
|
||||
cmd := &prepareCmd{
|
||||
out: out,
|
||||
errOut: errOut,
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
apiCall []*gomock.Call
|
||||
expectedError bool
|
||||
}{
|
||||
{
|
||||
name: "[prepare uninstall] network-qos reset failed",
|
||||
apiCall: []*gomock.Call{
|
||||
mockExec.EXPECT().CommandContext(gomock.Any(), utils.NetWorkCmdFile+" reset").Return("", fmt.Errorf("fake error")),
|
||||
},
|
||||
expectedError: true,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[prepare uninstall] update cni plugin conf failed",
|
||||
apiCall: []*gomock.Call{
|
||||
mockExec.EXPECT().CommandContext(gomock.Any(), utils.NetWorkCmdFile+" reset").Return("", nil),
|
||||
mockCniConf.EXPECT().DeleteCniPluginFromConfList(utils.DefaultCNIConfFile, utils.CNIPluginName).Return(fmt.Errorf("fake error")),
|
||||
},
|
||||
expectedError: true,
|
||||
},
|
||||
{
|
||||
name: "[prepare uninstall] uninstall successfully",
|
||||
apiCall: []*gomock.Call{
|
||||
mockExec.EXPECT().CommandContext(gomock.Any(), utils.NetWorkCmdFile+" reset").Return("", nil),
|
||||
mockCniConf.EXPECT().DeleteCniPluginFromConfList(utils.DefaultCNIConfFile, utils.CNIPluginName).Return(nil),
|
||||
},
|
||||
expectedError: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
actualErr := cmd.unInstall()
|
||||
assert.Equal(t, tc.expectedError, actualErr != nil, tc.name)
|
||||
}
|
||||
}
|
||||
|
||||
func TestInstall(t *testing.T) {
|
||||
mockController := gomock.NewController(t)
|
||||
defer mockController.Finish()
|
||||
|
||||
mockExec := mockexec.NewMockExecInterface(mockController)
|
||||
exec.SetExecutor(mockExec)
|
||||
mockCniConf := mockcni.NewMockCNIPluginsConfHandler(mockController)
|
||||
cni.SetCNIPluginConfHandler(mockCniConf)
|
||||
|
||||
cmd := &prepareCmd{}
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
onlineBandwidthWatermark string
|
||||
offlineLowBandwidth string
|
||||
offlineHighBandwidth string
|
||||
interval string
|
||||
apiCall []*gomock.Call
|
||||
expectedError bool
|
||||
}{
|
||||
{
|
||||
name: "[prepare install] network QoS enable && empty check-interval",
|
||||
interval: "",
|
||||
onlineBandwidthWatermark: "5000Mbps",
|
||||
offlineLowBandwidth: "2500Mbps",
|
||||
offlineHighBandwidth: "3500Mbps",
|
||||
apiCall: []*gomock.Call{
|
||||
mockExec.EXPECT().CommandContext(gomock.Any(), utils.NetWorkCmdFile+" set --online-bandwidth-watermark=5000Mbps "+
|
||||
"--offline-low-bandwidth=2500Mbps --offline-high-bandwidth=3500Mbps --check-interval=10000000").Return("", nil),
|
||||
mockCniConf.EXPECT().AddOrUpdateCniPluginToConfList(utils.DefaultCNIConfFile, utils.CNIPluginName, map[string]interface{}{
|
||||
"name": "network-qos",
|
||||
"type": "network-qos",
|
||||
"args": map[string]string{
|
||||
utils.NetWorkQoSCheckInterval: "10000000",
|
||||
utils.NodeColocationEnable: "true",
|
||||
utils.OnlineBandwidthWatermarkKey: "5000Mbps",
|
||||
utils.OfflineLowBandwidthKey: "2500Mbps",
|
||||
utils.OfflineHighBandwidthKey: "3500Mbps",
|
||||
},
|
||||
}).Return(nil),
|
||||
},
|
||||
expectedError: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[prepare install] network QoS enable",
|
||||
interval: "20000000",
|
||||
onlineBandwidthWatermark: "10000Mbps",
|
||||
offlineLowBandwidth: "2000Mbps",
|
||||
offlineHighBandwidth: "5000Mbps",
|
||||
|
||||
apiCall: []*gomock.Call{
|
||||
mockExec.EXPECT().CommandContext(gomock.Any(), utils.NetWorkCmdFile+" set --online-bandwidth-watermark=10000Mbps "+
|
||||
"--offline-low-bandwidth=2000Mbps --offline-high-bandwidth=5000Mbps --check-interval=20000000").Return("", nil),
|
||||
mockCniConf.EXPECT().AddOrUpdateCniPluginToConfList(utils.DefaultCNIConfFile, utils.CNIPluginName, map[string]interface{}{
|
||||
"name": "network-qos",
|
||||
"type": "network-qos",
|
||||
"args": map[string]string{
|
||||
utils.NetWorkQoSCheckInterval: "20000000",
|
||||
utils.NodeColocationEnable: "true",
|
||||
utils.OnlineBandwidthWatermarkKey: "10000Mbps",
|
||||
utils.OfflineLowBandwidthKey: "2000Mbps",
|
||||
utils.OfflineHighBandwidthKey: "5000Mbps",
|
||||
},
|
||||
}).Return(nil),
|
||||
},
|
||||
expectedError: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[prepare install] colocation set failed",
|
||||
interval: "",
|
||||
onlineBandwidthWatermark: "900Mbps",
|
||||
offlineLowBandwidth: "550Mbps",
|
||||
offlineHighBandwidth: "750Mbps",
|
||||
apiCall: []*gomock.Call{
|
||||
mockExec.EXPECT().CommandContext(gomock.Any(), utils.NetWorkCmdFile+" set --online-bandwidth-watermark=900Mbps "+
|
||||
"--offline-low-bandwidth=550Mbps --offline-high-bandwidth=750Mbps --check-interval=10000000").Return("", fmt.Errorf("fake error")),
|
||||
},
|
||||
expectedError: true,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[prepare install] update cni config failed",
|
||||
interval: "30000000",
|
||||
onlineBandwidthWatermark: "30000Mbps",
|
||||
offlineLowBandwidth: "5500Mbps",
|
||||
offlineHighBandwidth: "7500Mbps",
|
||||
apiCall: []*gomock.Call{
|
||||
mockExec.EXPECT().CommandContext(gomock.Any(), utils.NetWorkCmdFile+" set --online-bandwidth-watermark=30000Mbps "+
|
||||
"--offline-low-bandwidth=5500Mbps --offline-high-bandwidth=7500Mbps --check-interval=30000000").Return("", nil),
|
||||
mockCniConf.EXPECT().AddOrUpdateCniPluginToConfList(utils.DefaultCNIConfFile, utils.CNIPluginName, map[string]interface{}{
|
||||
"name": "network-qos",
|
||||
"type": "network-qos",
|
||||
"args": map[string]string{
|
||||
utils.NetWorkQoSCheckInterval: "30000000",
|
||||
utils.NodeColocationEnable: "true",
|
||||
utils.OnlineBandwidthWatermarkKey: "30000Mbps",
|
||||
utils.OfflineLowBandwidthKey: "5500Mbps",
|
||||
utils.OfflineHighBandwidthKey: "7500Mbps",
|
||||
},
|
||||
}).Return(fmt.Errorf("fake error")),
|
||||
},
|
||||
expectedError: true,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
actualErr := cmd.install(tc.onlineBandwidthWatermark, tc.offlineLowBandwidth, tc.offlineHighBandwidth, tc.interval)
|
||||
assert.Equal(t, tc.expectedError, actualErr != nil, tc.name, actualErr)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,85 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"strconv"
|
||||
|
||||
cilliumbpf "github.com/cilium/ebpf"
|
||||
"github.com/spf13/cobra"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
type resetCmd struct {
|
||||
out io.Writer
|
||||
errOut io.Writer
|
||||
}
|
||||
|
||||
func newResetCmd(out io.Writer, errOut io.Writer) *cobra.Command {
|
||||
reset := &resetCmd{
|
||||
out: out,
|
||||
errOut: errOut,
|
||||
}
|
||||
|
||||
return &cobra.Command{
|
||||
Use: "reset",
|
||||
Short: "Reset online-bandwidth-watermark/offline-low-bandwidth/offline-high-bandwidth values",
|
||||
Long: "Reset online-bandwidth-watermark/offline-low-bandwidth/offline-high-bandwidth values",
|
||||
Run: func(cmd *cobra.Command, args []string) {
|
||||
klog.InfoS("Network QoS command called", "command", "reset")
|
||||
err := reset.run()
|
||||
if err != nil {
|
||||
utils.Error(reset.errOut, cmd, err)
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (c *resetCmd) run() (err error) {
|
||||
throttlingConf, err := throttling.GetNetworkThrottlingConfig().GetThrottlingConfig()
|
||||
if err != nil {
|
||||
if errors.Is(err, cilliumbpf.ErrKeyNotExist) || os.IsNotExist(err) {
|
||||
fmt.Fprintf(c.out, "throttling config does not exist, reset successfully")
|
||||
klog.InfoS("Network QoS command called successfully, throttling config does not exist")
|
||||
return nil
|
||||
}
|
||||
return fmt.Errorf("failed to get throttling config: %v", err)
|
||||
}
|
||||
|
||||
throttlingConf, err = throttling.GetNetworkThrottlingConfig().CreateOrUpdateThrottlingConfig("", "1024Tibps", "1024Tibps",
|
||||
strconv.FormatUint(throttlingConf.Interval, 10))
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to update throttling config: %v", err)
|
||||
}
|
||||
|
||||
throttlingConfigBytes, err := json.Marshal(throttlingConf)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to marshal throttling conf: %v to json %v", throttlingConf, err)
|
||||
}
|
||||
|
||||
fmt.Fprintf(c.out, "throttling config reset: %s\n", throttlingConfigBytes)
|
||||
klog.InfoS("Network QoS command called successfully", "command", "reset", "throttling-conf", throttlingConfigBytes)
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,162 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"math"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/agiledragon/gomonkey/v2"
|
||||
cilliumbpf "github.com/cilium/ebpf"
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos/api"
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
mockthrottling "volcano.sh/volcano/pkg/networkqos/throttling/mocks"
|
||||
)
|
||||
|
||||
func TestResetCmdRun(t *testing.T) {
|
||||
mockController := gomock.NewController(t)
|
||||
defer mockController.Finish()
|
||||
|
||||
mockThr := mockthrottling.NewMockThrottlingConfig(mockController)
|
||||
throttling.SetNetworkThrottlingConfig(mockThr)
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
apiCall []*gomock.Call
|
||||
jsonMarshalErr error
|
||||
expectedOut string
|
||||
expectedErrOut string
|
||||
expectedOsExitCalled bool
|
||||
}{
|
||||
{
|
||||
name: "[reset] json Marshal failed",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 10000000,
|
||||
WaterLine: 1000,
|
||||
LowRate: 200,
|
||||
HighRate: 500,
|
||||
}, nil),
|
||||
mockThr.EXPECT().CreateOrUpdateThrottlingConfig("", "1024Tibps", "1024Tibps", "10000000").Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 10000000,
|
||||
WaterLine: math.MaxUint64,
|
||||
LowRate: math.MaxUint64,
|
||||
HighRate: math.MaxUint64,
|
||||
},
|
||||
nil),
|
||||
},
|
||||
jsonMarshalErr: fmt.Errorf("json marshal failed"),
|
||||
expectedErrOut: `execute command[reset] failed, error:failed to marshal throttling conf: &{18446744073709551615 10000000 18446744073709551615 18446744073709551615} to json json marshal failed` + "\n",
|
||||
expectedOsExitCalled: true,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[reset] reset existed conf successfully",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 10000000,
|
||||
WaterLine: 2000,
|
||||
LowRate: 800,
|
||||
HighRate: 1200,
|
||||
}, nil),
|
||||
mockThr.EXPECT().CreateOrUpdateThrottlingConfig("", "1024Tibps", "1024Tibps", "10000000").Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 10000000,
|
||||
WaterLine: math.MaxUint64,
|
||||
LowRate: math.MaxUint64,
|
||||
HighRate: math.MaxUint64,
|
||||
},
|
||||
nil),
|
||||
},
|
||||
expectedOut: `throttling config reset: {"online_bandwidth_watermark":18446744073709551615,"interval":10000000,"offline_low_bandwidth":18446744073709551615,"offline_high_bandwidth":18446744073709551615}` + "\n",
|
||||
},
|
||||
|
||||
{
|
||||
name: "[reset] cni conf ebpf map not exists",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(nil, cilliumbpf.ErrKeyNotExist),
|
||||
},
|
||||
expectedOut: `throttling config does not exist, reset successfully`,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[reset] cni conf ebpf map pinned file not exists",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(nil, os.ErrNotExist),
|
||||
},
|
||||
expectedOut: `throttling config does not exist, reset successfully`,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[reset] get cni conf map failed",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(nil, fmt.Errorf("failed to get ebpf map")),
|
||||
},
|
||||
expectedErrOut: `execute command[reset] failed, error:failed to get throttling config: failed to get ebpf map` + "\n",
|
||||
expectedOsExitCalled: true,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[reset] update cni conf map failed",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingConfig().Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 10000000,
|
||||
WaterLine: 5000,
|
||||
LowRate: 1000,
|
||||
HighRate: 3000,
|
||||
}, nil),
|
||||
mockThr.EXPECT().CreateOrUpdateThrottlingConfig("", "1024Tibps", "1024Tibps", "10000000").Return(nil, fmt.Errorf("update failed")),
|
||||
},
|
||||
expectedErrOut: `execute command[reset] failed, error:failed to update throttling config: update failed` + "\n",
|
||||
expectedOsExitCalled: true,
|
||||
},
|
||||
}
|
||||
|
||||
exitCalled := false
|
||||
resetPatch := gomonkey.NewPatches()
|
||||
resetPatch.ApplyFunc(os.Exit, func(code int) {
|
||||
exitCalled = true
|
||||
})
|
||||
|
||||
for _, tc := range testCases {
|
||||
exitCalled = false
|
||||
var jsonMarPatch *gomonkey.Patches
|
||||
if tc.jsonMarshalErr != nil {
|
||||
jsonMarPatch = gomonkey.NewPatches()
|
||||
jsonMarPatch.ApplyFunc(json.Marshal, func(v interface{}) ([]byte, error) {
|
||||
return nil, tc.jsonMarshalErr
|
||||
})
|
||||
}
|
||||
|
||||
out := &bytes.Buffer{}
|
||||
errOut := &bytes.Buffer{}
|
||||
cmd := newResetCmd(out, errOut)
|
||||
cmd.Execute()
|
||||
if jsonMarPatch != nil {
|
||||
jsonMarPatch.Reset()
|
||||
}
|
||||
assert.Equal(t, tc.expectedOsExitCalled, exitCalled, tc.name)
|
||||
assert.Equal(t, tc.expectedOut, out.String(), tc.name)
|
||||
assert.Equal(t, tc.expectedErrOut, errOut.String(), tc.name)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/cmd/network-qos/tools/options"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
var opt options.Options
|
||||
|
||||
// Execute adds all child commands to the root command and sets flags appropriately.
|
||||
// This is called by main.main(). It only needs to happen once to the rootCmd.
|
||||
func Execute() {
|
||||
err := utils.InitLog(utils.ToolCmdLogFilePath)
|
||||
if err != nil {
|
||||
fmt.Printf("failed fo init log: %v\n", err)
|
||||
}
|
||||
|
||||
err = NewRootCmd().Execute()
|
||||
if err != nil {
|
||||
klog.Flush()
|
||||
fmt.Fprintf(os.Stderr, "execute command failed, error:%v\n", err)
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
klog.Flush()
|
||||
os.Exit(0)
|
||||
}
|
||||
|
||||
func NewRootCmd() *cobra.Command {
|
||||
cmd := &cobra.Command{
|
||||
Use: "network-qos",
|
||||
Short: "Network QoS",
|
||||
Long: "Network QoS",
|
||||
}
|
||||
|
||||
cmd.AddCommand(newSetCmd(os.Stdout, os.Stderr))
|
||||
cmd.AddCommand(newGetCmd(os.Stdout, os.Stderr))
|
||||
cmd.AddCommand(newResetCmd(os.Stdout, os.Stderr))
|
||||
cmd.AddCommand(newStatusCmd(os.Stdout, os.Stderr))
|
||||
cmd.AddCommand(newPrepareCmd(os.Stdout, os.Stderr))
|
||||
cmd.AddCommand(newVersionCmd(os.Stdout, os.Stderr))
|
||||
return cmd
|
||||
}
|
||||
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/cmd/network-qos/tools/options"
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
type setCmd struct {
|
||||
out io.Writer
|
||||
errOut io.Writer
|
||||
}
|
||||
|
||||
func newSetCmd(out io.Writer, errOut io.Writer) *cobra.Command {
|
||||
set := &setCmd{
|
||||
out: out,
|
||||
errOut: errOut,
|
||||
}
|
||||
|
||||
cmd := &cobra.Command{
|
||||
Use: "set",
|
||||
Short: "Add/Update online-bandwidth-watermark/offline-low-bandwidth/offline-high-bandwidth/check-interval values",
|
||||
Long: "Add/Update online-bandwidth-watermark/offline-low-bandwidth/offline-high-bandwidth/check-interval values",
|
||||
Run: func(cmd *cobra.Command, args []string) {
|
||||
klog.InfoS("Network QoS command called", "command", "set")
|
||||
err := set.run(opt)
|
||||
if err != nil {
|
||||
utils.Error(set.errOut, cmd, err)
|
||||
}
|
||||
},
|
||||
}
|
||||
(&opt).AddFlags(cmd)
|
||||
return cmd
|
||||
}
|
||||
|
||||
func (c *setCmd) run(opt options.Options) (err error) {
|
||||
throttlingConf, err := throttling.GetNetworkThrottlingConfig().CreateOrUpdateThrottlingConfig(opt.OnlineBandwidthWatermark,
|
||||
opt.OfflineLowBandwidth, opt.OfflineHighBandwidth, opt.CheckoutInterval)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create/update throttling config: %v", err)
|
||||
}
|
||||
|
||||
throttlingConfigBytes, err := json.Marshal(throttlingConf)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create/update throttling config: %v", err)
|
||||
}
|
||||
|
||||
fmt.Fprintf(c.out, "throttling config set: %s\n", throttlingConfigBytes)
|
||||
klog.InfoS("Network QoS command called successfully", "command", "set", "throttling-conf", throttlingConfigBytes)
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,140 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/agiledragon/gomonkey/v2"
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"volcano.sh/volcano/cmd/network-qos/tools/options"
|
||||
"volcano.sh/volcano/pkg/networkqos/api"
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
mockthrottling "volcano.sh/volcano/pkg/networkqos/throttling/mocks"
|
||||
)
|
||||
|
||||
func TestSetCmdRun(t *testing.T) {
|
||||
mockController := gomock.NewController(t)
|
||||
defer mockController.Finish()
|
||||
|
||||
mockThr := mockthrottling.NewMockThrottlingConfig(mockController)
|
||||
throttling.SetNetworkThrottlingConfig(mockThr)
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
opt options.Options
|
||||
apiCall []*gomock.Call
|
||||
jsonMarshalErr error
|
||||
expectedOut string
|
||||
expectedErrOut string
|
||||
expectedOsExitCalled bool
|
||||
}{
|
||||
{
|
||||
name: "[set] json Marshal failed",
|
||||
opt: options.Options{
|
||||
CheckoutInterval: "100",
|
||||
OnlineBandwidthWatermark: "100MB",
|
||||
OfflineLowBandwidth: "30MB",
|
||||
OfflineHighBandwidth: "50MB",
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().CreateOrUpdateThrottlingConfig("100MB", "30MB", "50MB", "100").Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 10000000,
|
||||
WaterLine: 100 * 1024 * 1024,
|
||||
LowRate: 30 * 1024 * 1024,
|
||||
HighRate: 50 * 1024 * 1024,
|
||||
},
|
||||
nil),
|
||||
},
|
||||
jsonMarshalErr: fmt.Errorf("json marshal failed"),
|
||||
expectedErrOut: `execute command[set] failed, error:failed to create/update throttling config: json marshal failed` + "\n",
|
||||
expectedOsExitCalled: true,
|
||||
},
|
||||
|
||||
{
|
||||
name: "[set] set throttling conf map successfully",
|
||||
opt: options.Options{
|
||||
CheckoutInterval: "200",
|
||||
OnlineBandwidthWatermark: "1000MB",
|
||||
OfflineLowBandwidth: "200MB",
|
||||
OfflineHighBandwidth: "500MB",
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().CreateOrUpdateThrottlingConfig("1000MB", "200MB", "500MB", "200").Return(&api.EbpfNetThrottlingConfig{
|
||||
Interval: 20000000,
|
||||
WaterLine: 1000 * 1024 * 1024,
|
||||
LowRate: 200 * 1024 * 1024,
|
||||
HighRate: 500 * 1024 * 1024,
|
||||
},
|
||||
nil),
|
||||
},
|
||||
expectedOut: `throttling config set: {"online_bandwidth_watermark":1048576000,"interval":20000000,"offline_low_bandwidth":209715200,"offline_high_bandwidth":524288000}` + "\n",
|
||||
},
|
||||
|
||||
{
|
||||
name: "[set] set throttling conf map failed",
|
||||
opt: options.Options{
|
||||
CheckoutInterval: "300",
|
||||
OnlineBandwidthWatermark: "800MB",
|
||||
OfflineLowBandwidth: "250MB",
|
||||
OfflineHighBandwidth: "350MB",
|
||||
},
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().CreateOrUpdateThrottlingConfig("800MB", "250MB", "350MB", "300").Return(nil,
|
||||
fmt.Errorf("failed to update conf map")),
|
||||
},
|
||||
expectedErrOut: `execute command[set] failed, error:failed to create/update throttling config: failed to update conf map` + "\n",
|
||||
expectedOsExitCalled: true,
|
||||
},
|
||||
}
|
||||
|
||||
exitCalled := false
|
||||
setPatch := gomonkey.NewPatches()
|
||||
setPatch.ApplyFunc(os.Exit, func(code int) {
|
||||
exitCalled = true
|
||||
})
|
||||
defer setPatch.Reset()
|
||||
|
||||
for _, tc := range testCases {
|
||||
exitCalled = false
|
||||
var jsonMarshalPatch *gomonkey.Patches
|
||||
if tc.jsonMarshalErr != nil {
|
||||
jsonMarshalPatch = gomonkey.NewPatches()
|
||||
jsonMarshalPatch.ApplyFunc(json.Marshal, func(v interface{}) ([]byte, error) {
|
||||
return nil, tc.jsonMarshalErr
|
||||
})
|
||||
}
|
||||
|
||||
out := &bytes.Buffer{}
|
||||
errOut := &bytes.Buffer{}
|
||||
opt = tc.opt
|
||||
cmd := newSetCmd(out, errOut)
|
||||
cmd.Execute()
|
||||
if jsonMarshalPatch != nil {
|
||||
jsonMarshalPatch.Reset()
|
||||
}
|
||||
assert.Equal(t, tc.expectedOsExitCalled, exitCalled, tc.name)
|
||||
assert.Equal(t, tc.expectedOut, out.String(), tc.name)
|
||||
assert.Equal(t, tc.expectedErrOut, errOut.String(), tc.name)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
|
||||
cilliumbpf "github.com/cilium/ebpf"
|
||||
"github.com/spf13/cobra"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
type statusCmd struct {
|
||||
out io.Writer
|
||||
errOut io.Writer
|
||||
}
|
||||
|
||||
func newStatusCmd(out io.Writer, errOut io.Writer) *cobra.Command {
|
||||
status := &statusCmd{
|
||||
out: out,
|
||||
errOut: errOut,
|
||||
}
|
||||
|
||||
return &cobra.Command{
|
||||
Use: "status",
|
||||
Short: "Print network QoS status",
|
||||
Long: "Print network QoS status",
|
||||
Run: func(cmd *cobra.Command, args []string) {
|
||||
klog.InfoS("Network QoS command called", "command", "status")
|
||||
err := status.run()
|
||||
if err != nil {
|
||||
utils.Error(status.errOut, cmd, err)
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (c *statusCmd) run() (err error) {
|
||||
throttlingStatus, err := throttling.GetNetworkThrottlingConfig().GetThrottlingStatus()
|
||||
if err != nil {
|
||||
if errors.Is(err, cilliumbpf.ErrKeyNotExist) || os.IsNotExist(err) {
|
||||
return fmt.Errorf("failed to get throttling status: %v, network qos has not been initialized, please enable network qos first", err)
|
||||
}
|
||||
return fmt.Errorf("failed to get throttling status: %v", err)
|
||||
}
|
||||
|
||||
throttlingStatusBytes, err := json.Marshal(throttlingStatus)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to marshal throttling status %v to json: %v", throttlingStatus, err)
|
||||
}
|
||||
|
||||
fmt.Fprintf(c.out, "%s: %s\n", "throttling status", throttlingStatusBytes)
|
||||
klog.InfoS("Network QoS command called successfully", "command", "status", "status", throttlingStatusBytes)
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,131 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/agiledragon/gomonkey/v2"
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos/api"
|
||||
"volcano.sh/volcano/pkg/networkqos/throttling"
|
||||
mockthrottling "volcano.sh/volcano/pkg/networkqos/throttling/mocks"
|
||||
)
|
||||
|
||||
func TestStatusCmdExecute(t *testing.T) {
|
||||
mockController := gomock.NewController(t)
|
||||
defer mockController.Finish()
|
||||
|
||||
mockThr := mockthrottling.NewMockThrottlingConfig(mockController)
|
||||
throttling.SetNetworkThrottlingConfig(mockThr)
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
apiCall []*gomock.Call
|
||||
jsonMarshalErr error
|
||||
expectedOut string
|
||||
expectedErrOut string
|
||||
expectedOsExitCalled bool
|
||||
}{
|
||||
{
|
||||
name: "[status] json Marshal failed",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingStatus().Return(&api.EbpfNetThrottling{
|
||||
TLast: 65923918434500,
|
||||
Rate: 50000000,
|
||||
TXBytes: 20000,
|
||||
OnlineTXBytes: 20000,
|
||||
TStart: 88736282681743,
|
||||
EbpfNetThrottlingStatus: api.EbpfNetThrottlingStatus{
|
||||
CheckTimes: 300,
|
||||
HighTimes: 100,
|
||||
LowTimes: 200,
|
||||
OnlinePKTs: 100,
|
||||
OfflinePKTs: 100,
|
||||
},
|
||||
}, nil),
|
||||
},
|
||||
jsonMarshalErr: fmt.Errorf("json marshal failed"),
|
||||
expectedErrOut: `execute command[status] failed, error:failed to marshal throttling status &{65923918434500 50000000 20000 20000 88736282681743 {300 100 200 100 100 0 0 0}} to json: json marshal failed` + "\n",
|
||||
expectedOsExitCalled: true,
|
||||
},
|
||||
{
|
||||
name: "[status] get conf successfully",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingStatus().Return(&api.EbpfNetThrottling{
|
||||
TLast: 65923918434483,
|
||||
Rate: 60000000,
|
||||
TXBytes: 10000,
|
||||
OnlineTXBytes: 10000,
|
||||
TStart: 88736282681743,
|
||||
EbpfNetThrottlingStatus: api.EbpfNetThrottlingStatus{
|
||||
CheckTimes: 1000,
|
||||
HighTimes: 500,
|
||||
LowTimes: 500,
|
||||
OnlinePKTs: 500,
|
||||
OfflinePKTs: 500,
|
||||
},
|
||||
}, nil),
|
||||
},
|
||||
expectedOut: `throttling status: {"latest_offline_packet_send_time":65923918434483,"offline_bandwidth_limit":60000000,"offline_tx_bytes":10000,"online_tx_bytes":10000,"latest_check_time":88736282681743,"check_times":1000,"high_times":500,"low_times":500,"online_tx_packages":500,"offline_tx_packages":500,"offline_prio":0,"latest_online_bandwidth":0,"latest_offline_bandwidth":0}` + "\n",
|
||||
},
|
||||
|
||||
{
|
||||
name: "[status] get conf failed",
|
||||
apiCall: []*gomock.Call{
|
||||
mockThr.EXPECT().GetThrottlingStatus().Return(nil, fmt.Errorf("request failed")),
|
||||
},
|
||||
expectedErrOut: `execute command[status] failed, error:failed to get throttling status: request failed` + "\n",
|
||||
expectedOsExitCalled: true,
|
||||
},
|
||||
}
|
||||
|
||||
exitCalled := false
|
||||
statusPatch := gomonkey.NewPatches()
|
||||
statusPatch.ApplyFunc(os.Exit, func(code int) {
|
||||
exitCalled = true
|
||||
})
|
||||
defer statusPatch.Reset()
|
||||
|
||||
for _, tc := range testCases {
|
||||
exitCalled = false
|
||||
var jsonMarPatch *gomonkey.Patches
|
||||
if tc.jsonMarshalErr != nil {
|
||||
jsonMarPatch = gomonkey.NewPatches()
|
||||
jsonMarPatch.ApplyFunc(json.Marshal, func(v interface{}) ([]byte, error) {
|
||||
return nil, tc.jsonMarshalErr
|
||||
})
|
||||
}
|
||||
|
||||
out := &bytes.Buffer{}
|
||||
errOut := &bytes.Buffer{}
|
||||
cmd := newStatusCmd(out, errOut)
|
||||
cmd.Execute()
|
||||
if jsonMarPatch != nil {
|
||||
jsonMarPatch.Reset()
|
||||
}
|
||||
assert.Equal(t, tc.expectedOsExitCalled, exitCalled, tc.name)
|
||||
assert.Equal(t, tc.expectedOut, out.String(), tc.name)
|
||||
assert.Equal(t, tc.expectedErrOut, errOut.String(), tc.name)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,36 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package tools
|
||||
|
||||
import (
|
||||
"io"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
|
||||
"volcano.sh/volcano/pkg/version"
|
||||
)
|
||||
|
||||
func newVersionCmd(out io.Writer, errOut io.Writer) *cobra.Command {
|
||||
return &cobra.Command{
|
||||
Use: "version",
|
||||
Short: "Print version information",
|
||||
Long: "Print version information",
|
||||
Run: func(cmd *cobra.Command, args []string) {
|
||||
version.PrintVersionAndExit()
|
||||
},
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,456 @@
|
|||
## Motivation
|
||||
|
||||
Volcano now supports unified scheduling of microservices and batch workloads, it's compatible with the default scheduling algorithms of kubernetes, and provides rich scheduling policies for microservices and batch jobs, so it is suitable for unified scheduling of different types of jobs in colocation scenarios, and improves resource utilization simultaneously.
|
||||
|
||||
## Architecture
|
||||
|
||||
Volcano agent supports cloud-native online and offline workloads colocation. It runs as a daemonset in the kubernetes cluster, and is committed to improving the resource utilization of the kubernetes cluster and meanwhile ensures the Qos of online workloads through CPU/Memory/Network resource isolation and suppression.
|
||||
|
||||

|
||||
|
||||
### Qos Model
|
||||
|
||||
| Qos | Typical Usage Scenario |
|
||||
| :---------------------------: | :------------------------------------------------------: |
|
||||
| LC(Latency Critical) | Exclusive CPU for latency critical core online workloads |
|
||||
| HLS(Highly Latency Sensitive) | Extremely latency sensitive online workloads |
|
||||
| LS(Latency Sensitive) | Latency sensitive approximate online workloads |
|
||||
| BE(Best Effort) | Offline AI/Big Data workloads, tolerable evictions |
|
||||
|
||||
You can set different qos levels to identify workloads of different priorities, and there is no seperate `priorityClass` definition to be consistent with k8s.
|
||||
|
||||
### Features
|
||||
|
||||
- Unified scheduling for all kinds of workloads including microservices and batch jobs.
|
||||
- Qos based resource model for online and offline workloads.
|
||||
- CPU burst: Allow containers to temporarily exceed the CPU limit to avoid throttling at critical moments.
|
||||
- Dynamic resource oversubscription: Dynamically calculate the resources that can be oversold based on the real-time CPU/Memory utilization of the node, and oversold resources can be used by offline workloads.
|
||||
- Network bandwidth isolation:Supports ingress network bandwidth limitation of the entire machine to ensure network usage for online workloads.
|
||||
|
||||
## Quick start
|
||||
|
||||
### Prepare
|
||||
|
||||
Follow the [installation guide](https://github.com/volcano-sh/volcano?tab=readme-ov-file#quick-start-guide) to install volcano first, and then install volcano agent via kubectl, this way is both available for x86_64 and arm64 architecture.
|
||||
|
||||
```shell
|
||||
kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/master/installer/volcano-agent-development.yaml
|
||||
```
|
||||
|
||||
Check volcano components including agent runs successfully.
|
||||
|
||||
```shell
|
||||
$ kubectl get po -n volcano-system
|
||||
NAME READY STATUS RESTARTS AGE
|
||||
volcano-admission-76bd985b56-fnpjg 1/1 Running 0 3d
|
||||
volcano-admission-init-wmxc7 0/1 Completed 0 3d
|
||||
volcano-agent-q85jn 1/1 Running 0 3d
|
||||
volcano-controllers-7655bb499f-gpg9l 1/1 Running 0 3d
|
||||
volcano-scheduler-6bf4759c45-c666z 1/1 Running 0 3d
|
||||
```
|
||||
|
||||
Enable node level colocation by setting lable volcano.sh/oversubscription=true and volcano.sh/colocation=true.
|
||||
|
||||
```
|
||||
$ kubectl label node $node volcano.sh/oversubscription=true # replace $node with real node name in your kubernetes cluster.
|
||||
|
||||
$ kubectl label node $node volcano.sh/colocation=true # replace $node with real node name in your kubernetes cluster.
|
||||
```
|
||||
|
||||
### CPU burst tutorial
|
||||
|
||||
This example will demonstrate how to use CPU burst and what benefit it can bring.
|
||||
|
||||
#### Enable cpu burst by set pod annotation
|
||||
|
||||
Run a nginx deployment with cluster ip service exposed, pod with annotations `volcano.sh/enable-quota-burst: "true"` indicates pod cpu
|
||||
|
||||
burst is enabled.
|
||||
|
||||
```yaml
|
||||
apiVersion: apps/v1
|
||||
kind: Deployment
|
||||
metadata:
|
||||
name: nginx
|
||||
namespace: default
|
||||
spec:
|
||||
replicas: 1
|
||||
selector:
|
||||
matchLabels:
|
||||
app: nginx
|
||||
template:
|
||||
metadata:
|
||||
labels:
|
||||
app: nginx
|
||||
annotations:
|
||||
volcano.sh/enable-quota-burst: "true" # pod enabled cpu burst
|
||||
spec:
|
||||
containers:
|
||||
- name: container-1
|
||||
image: nginx:latest
|
||||
resources:
|
||||
limits:
|
||||
cpu: "2"
|
||||
requests:
|
||||
cpu: "1"
|
||||
---
|
||||
apiVersion: v1
|
||||
kind: Service
|
||||
metadata:
|
||||
name: nginx
|
||||
namespace: default
|
||||
labels:
|
||||
app: nginx
|
||||
spec:
|
||||
selector:
|
||||
app: nginx
|
||||
ports:
|
||||
- name: http
|
||||
targetPort: 80
|
||||
port: 80
|
||||
protocol: TCP
|
||||
type: ClusterIP
|
||||
```
|
||||
|
||||
#### Perform stress
|
||||
|
||||
Perform stress with `wrk` to nginx app.
|
||||
|
||||
```shell
|
||||
wrk -H "Accept-Encoding: deflate, gzip" -t 2 -c 8 -d 120 --latency --timeout 2s http://$(kubectl get svc nginx -o jsonpath='{.spec.clusterIP}')
|
||||
```
|
||||
|
||||
#### Check cpu burst
|
||||
|
||||
Check container's cpu throttle state of pod, we can see that `nr_bursts` and `burst_time` is not 0, `nr_throttled` and `throttled_time` is a small value, which indicates pod has used burst cpu quota.
|
||||
|
||||
```shell
|
||||
$ cat /sys/fs/cgroup/cpu/kubepods/burstable/podd2988e14-83bc-4d3d-931a-59f8a3174396/cpu.stat # replace nginx pod uid in your kubernetes cluster.
|
||||
nr_periods 1210
|
||||
nr_throttled 9
|
||||
throttled_time 193613865
|
||||
nr_bursts 448
|
||||
burst_time 6543701690
|
||||
```
|
||||
|
||||
If we set pod's annotations `volcano.sh/enable-quota-burst=false`(disable pod cpu burst) and perform another stress, `nr_throttled` and `throttled_time` will be a relatively large value, which indicates pod cpu is throttled strictly, `nr_bursts` and `burst_time` is 0, indicates that pod cpu burst not happened.
|
||||
|
||||
```shell
|
||||
$ cat /sys/fs/cgroup/cpu/kubepods/burstable/podeeb542c6-b667-4da4-9ac9-86ced4e93fbb/cpu.stat #replace nginx pod uid in your kubernetes cluster.
|
||||
nr_periods 1210
|
||||
nr_throttled 488
|
||||
throttled_time 10125826283
|
||||
nr_bursts 0
|
||||
burst_time 0
|
||||
```
|
||||
|
||||
#### Limitation
|
||||
|
||||
CPU burst relies on capabilities provided by the linux kernel, this feature only works with host upstream linux kernel >=5.14 and some Linux Distribution like OpenEuler 22.03 SP2 or higher version.
|
||||
|
||||
### Dynamic resource oversubscription tutorial
|
||||
|
||||
This example will demonstrate the resource overoversubscription capability on node, and shows the suppression and eviction mechanism when node is suffering from pressure. The node flavor is 8 core cpu and 16GB memory.
|
||||
|
||||
#### Check node oversubscription resoures
|
||||
|
||||
Node oversubscription resources are calculated by node allocatable resources sub actual resource usage, oversubscription resources include cpu and memory and is represented by `kubernetes.io/batch-cpu` and `kubernetes.io/batch-memory` respectively, and reported as extended resources to node.Allocatable filed. Online workloads use noraml resources and offline workloads use oversubscription resources so we can improve pod deployment density and resource utilization.
|
||||
|
||||
```shell
|
||||
$ kubectl describe node $node # replace $node with real node name in your kubernetes cluster.
|
||||
Allocatable:
|
||||
cpu: 8
|
||||
ephemeral-storage: 33042054704
|
||||
hugepages-1Gi: 0
|
||||
hugepages-2Mi: 0
|
||||
kubernetes.io/batch-cpu: 7937 # oversubscription resource cpu, unit is milli cpu
|
||||
kubernetes.io/batch-memory: 14327175770 # oversubscription resource memory, utit is byte
|
||||
memory: 15754924Ki
|
||||
pods: 110
|
||||
```
|
||||
|
||||
#### Deploy online and offline workloads
|
||||
|
||||
Online workloads are identified by setting annotation `volcano.sh/qos-level: "LC" or` `volcano.sh/qos-level: "HLS"` or `volcano.sh/qos-level: "LS"`.Offline workloads are identified by setting annotation `volcano.sh/qos-level: "BE"`, it can only use oversubscrption resources(kubernetes.io/batch-cpu and kubernetes.io/batch-memory). We use an image with stress tool of online workload to simulate online workloads business pressure rises, if you can not access the image, you can also replace it with other image that has stress tool.
|
||||
|
||||
```yaml
|
||||
# online workload
|
||||
apiVersion: apps/v1
|
||||
kind: Deployment
|
||||
metadata:
|
||||
name: online-demo
|
||||
namespace: default
|
||||
spec:
|
||||
replicas: 1
|
||||
selector:
|
||||
matchLabels:
|
||||
app: online-demo
|
||||
template:
|
||||
metadata:
|
||||
labels:
|
||||
app: online-demo
|
||||
annotations:
|
||||
volcano.sh/qos-level: "HLS" # identify online workloads
|
||||
spec:
|
||||
containers:
|
||||
- name: container-1
|
||||
image: polinux/stress
|
||||
imagePullPolicy: IfNotPresent
|
||||
command: ["stress", "--cpu", "7"] # perform cpu stress
|
||||
resources:
|
||||
requests:
|
||||
cpu: 2
|
||||
---
|
||||
# offline workload
|
||||
apiVersion: apps/v1
|
||||
kind: Deployment
|
||||
metadata:
|
||||
name: offline-demo
|
||||
namespace: default
|
||||
spec:
|
||||
replicas: 1
|
||||
selector:
|
||||
matchLabels:
|
||||
app: offline-demo
|
||||
template:
|
||||
metadata:
|
||||
labels:
|
||||
app: offline-demo
|
||||
annotations:
|
||||
volcano.sh/qos-level: "BE" # identify offline workloads
|
||||
spec:
|
||||
containers:
|
||||
- name: container-1
|
||||
image: nginx:latest
|
||||
resources:
|
||||
requests:
|
||||
kubernetes.io/batch-cpu: 4000 # 4 core cpu
|
||||
kubernetes.io/batch-memory: 10737418240 # 10Gi memory
|
||||
```
|
||||
|
||||
Make sure online and offline workloads are running.
|
||||
|
||||
```shell
|
||||
$ kubectl get po
|
||||
NAME READY STATUS RESTARTS AGE
|
||||
offline-demo-f59758bb-vlbp7 1/1 Running 0 6s
|
||||
online-demo-9f9bbdb58-fljzs 1/1 Running 0 6s
|
||||
```
|
||||
|
||||
#### Eviction happens when node has pressure
|
||||
|
||||
Online workloads Qos is guaranteed by volcano agent and host OS, volcano agent detects node resource utilization in real time, and will evict offline workloads when node resource utilization exceeds the threshold, for CPU resource, it's 80% by default, we perform 7 core cpu stress to online workload, we can check event and it showed that offilne worload is evicted after about 1 minutes later.
|
||||
|
||||
```shell
|
||||
$ kubectl get event | grep Evicted
|
||||
69s Warning Evicted pod/offline-demo-785cff7f58-gwqwc Evict offline pod due to cpu resource pressure
|
||||
```
|
||||
|
||||
We can also find that oversubscription reduced when node has pressure.
|
||||
|
||||
```shell
|
||||
$ kubectl describe node $node # replace $node with real node name in your kubernetes cluster.
|
||||
Allocatable:
|
||||
cpu: 8
|
||||
ephemeral-storage: 33042054704
|
||||
hugepages-1Gi: 0
|
||||
hugepages-2Mi: 0
|
||||
kubernetes.io/batch-cpu: 978 # oversubscription resource of cpu reduced.
|
||||
kubernetes.io/batch-memory: 14310391443
|
||||
memory: 15754924Ki
|
||||
pods: 110
|
||||
```
|
||||
|
||||
Volcano agent will also add eviction taint to current node when eviction happened to avoid new workloads continuously scheduled to the node and put additional pressure on current node. We can find that new created offline pod is pening because of eviction taint.
|
||||
|
||||
```shell
|
||||
$ kubectl get po
|
||||
NAME READY STATUS RESTARTS AGE
|
||||
offline-demo-f59758bb-kwb54 0/1 Pending 0 58s
|
||||
online-demo-9f9bbdb58-54fnx 1/1 Running 0 2m1s
|
||||
|
||||
$ kubectl describe po offline-demo-f59758bb-kwb54
|
||||
Events:
|
||||
Type Reason Age From Message
|
||||
---- ------ ---- ---- -------
|
||||
Warning FailedScheduling 69s default-scheduler 0/1 nodes are available: 1 node(s) had taint {volcano.sh/offline-job-evicting: }, that the pod didn't tolerate.
|
||||
```
|
||||
|
||||
And if we stop online workload to release pressure then the eviction taint will be removed and pod can be scheduled normally.
|
||||
|
||||
#### Limitation
|
||||
|
||||
Volcano agent defines a Qos resource model for online and offline workloads, and provides application level guarantee(eviction when node has pressure) for online workloads Qos guarantee. The OS level cpu and memory isolation and suppress are guaranteed by host kernal, and currently volcano agent only adapted to openEuler 22.03 SP2 and higher version, please make sure that you are using the correct OS type and version.
|
||||
|
||||
### Network bandwidth isolation tutorial
|
||||
|
||||
The bandwidth usage of offline workloads will be limited when online workloads require more bandwidth. There are three watermarks parameters of network bandwidth isolation.
|
||||
|
||||
| Watermark | Description | Default Value |
|
||||
| :-------------------------------: | :----------------------------------------------------------: | :-----------: |
|
||||
| `onlineBandwidthWatermarkPercent` | The ratio of online bandwidth watermark value to the base bandwidth of node:<br/>onlineBandwidthWatermark value = node base bandwidth * `onlineBandwidthWatermarkPercent`/100 | 80 |
|
||||
| `offlineHighBandwidthPercent` | The ratio of offline high bandwidth watermark value to the base bandwidth of node:<br/>offlineHighBandwidth value = node base bandwidth * `offlineHighBandwidthPercent`/100<br/>It represents the upper limit of bandwidth that can be used by offline workloads when the online workloads use bandwidth ratio less than `onlineBandwidthWatermarkPercent`,eg: node bandwidth=100Mbps, `onlineBandwidthWatermarkPercent` =80 and `offlineHighBandwidthPercent`=40, when online workloads use bandwidth less than 100Mbps\*0.8=80Mbps, then the offline workloads can use at most 100Mbps\*40=40Mbps bandwidth. | 40 |
|
||||
| `offlineLowBandwidthPercent` | The ratio of offline low bandwidth watermark value to the base bandwidth of node:<br/>offlineLowBandwidth value = node base bandwidth * `offlineLowBandwidthPercent`/100<br/>It represents the upper limit of bandwidth that can be used by offline workloads when the online workloads use bandwidth ratio more than `onlineBandwidthWatermarkPercent`,eg: node bandwidth=100Mbps, `onlineBandwidthWatermarkPercent` =80 and `offlineLowBandwidthPercent`=10, when online workloads use bandwidth more than 100Mbps\*0.8=80Mbps, then the offline workloads can use at most 100Mbps\*10=10Mbps bandwidth. | 10 |
|
||||
|
||||
This picture below shows the bandwidth that online and offline workloads actually can use when they compete for bandwidth resources on one node.
|
||||
|
||||

|
||||
|
||||
#### Set network bandwidth
|
||||
|
||||
This example will demonstrate the suppression of the entire network bandwidth of an offline workload by online workload, and we use`iperf` tool to simulate network ingress bandwidth traffic of both online and offline workloads.
|
||||
|
||||
Add an annotation `volcano.sh/network-bandwidth-rate` on all nodes to specify the network bandwidth rate, the value in the example is 1000Mbps, please set an actual value according to your environment, and please replace `$node` with actual node name in your environment.
|
||||
|
||||
```shell
|
||||
$ kubectl annotate node $node_name volcano.sh/network-bandwidth-rate=1000
|
||||
```
|
||||
|
||||
#### Run online and offline workloads
|
||||
|
||||
Run an online deployment with below container command. Please replace the `$node_ip` with a node ip that pod can access in your environment and also make sure iperf tool is installed in container image. And please also start iperf server on `$node_ip` with command `iperf -s` to make sure pod can access the iperf server.
|
||||
|
||||
```shell
|
||||
- command
|
||||
- /bin/sh
|
||||
- c
|
||||
- iperf -c $node_ip -i 1 -t 30 -f mb; echo finished...; sleep 1000000
|
||||
```
|
||||
|
||||
And set the following annotation to identity the online workload.
|
||||
|
||||
```yaml
|
||||
annotations:
|
||||
volcano.sh/qos-level: "HLS" # identify online workloads
|
||||
```
|
||||
|
||||
Run an offline deployment with podTemplate specified with the following annotation
|
||||
|
||||
```yaml
|
||||
annotations:
|
||||
volcano.sh/qos-level: "BE" # identify offline workloads
|
||||
```
|
||||
|
||||
and also specify the container command like online deployment.
|
||||
|
||||
```shell
|
||||
- command
|
||||
- /bin/sh
|
||||
- c
|
||||
- iperf -c $node_ip -i 1 -t 30 -f mb; echo finished...; sleep 1000000
|
||||
```
|
||||
|
||||
#### Check logs
|
||||
|
||||
Check online and offine pod logs
|
||||
|
||||
online pod log:
|
||||
|
||||
```shell
|
||||
Connecting to host 192.168.2.30, port 5201
|
||||
[ 5] local 192.168.2.115 port 58492 connected to 192.168.2.30 port 5201
|
||||
[ ID] Interval Transfer Bandwidth
|
||||
[ 5] 0.00-1.00 sec 118 MBytes 990 Mbits/sec
|
||||
[ 5] 1.00-2.00 sec 106 MBytes 889 Mbits/sec
|
||||
[ 5] 2.00-3.00 sec 107 MBytes 897 Mbits/sec
|
||||
[ 5] 3.00-4.00 sec 107 MBytes 903 Mbits/sec
|
||||
[ 5] 4.00-5.00 sec 107 MBytes 899 Mbits/sec
|
||||
[ 5] 5.00-6.00 sec 107 MBytes 902 Mbits/sec
|
||||
[ 5] 6.00-7.00 sec 705 MBytes 884 Mbits/sec
|
||||
...
|
||||
```
|
||||
|
||||
offline pod log:
|
||||
|
||||
```shell
|
||||
Connecting to host 192.168.2.30, port 5201
|
||||
[ 5] local 192.168.2.115 port 44362 connected to 192.168.2.30 port 5201
|
||||
[ ID] Interval Transfer Bandwidth
|
||||
[ 5] 0.00-1.00 sec 8 MBytes 70 Mbits/sec
|
||||
[ 5] 1.00-2.00 sec 12 MBytes 102 Mbits/sec
|
||||
[ 5] 2.00-3.00 sec 11 MBytes 98 Mbits/sec
|
||||
[ 5] 3.00-4.00 sec 11 MBytes 99 Mbits/sec
|
||||
[ 5] 4.00-5.00 sec 11 MBytes 99 Mbits/sec
|
||||
[ 5] 5.00-6.00 sec 11 MBytes 97 Mbits/sec
|
||||
[ 5] 6.00-7.00 sec 11 MBytes 98 Mbits/sec
|
||||
...
|
||||
```
|
||||
|
||||
You can see that offline pod can use almost 10% bandwidth when online pod use more bandwidth resources than `onlineBandwidthWatermarkPercent` of the whole node.
|
||||
|
||||
## Advanced settings
|
||||
|
||||
### Feature switch
|
||||
|
||||
Colocation feature has a unified switch on node, node has label volcano.sh/oversubscription=true or volcano.sh/colocation=true indicates that coloation is enabled. You can remove the two labels to disable all colocation features. And all colocation features take effect when node has these lables.
|
||||
|
||||
- If you only want to use online and offline workloads colocation but not resource oversubscription, you just need to set the node label volcano.sh/colocation="true".
|
||||
- If you want to use colocation and resource oversubscription, you should set node label volcano.sh/oversubscription=true.
|
||||
|
||||
The configMap `volcano-agent-configuration` of namespace `volcano-system` by default holds all the configurations of volcano agent.
|
||||
|
||||
Every colocation feature CPU burst/Dynamic resource oversubscription/Network bandwidth isolation has a separate switch, you can enable/disable each of them by modify configMap `volcano-agent-configuration` of namespace,
|
||||
|
||||
enable filed value true means enable CPU burst, false means disable it.
|
||||
|
||||
```json
|
||||
"cpuBurstConfig":{
|
||||
"enable": true
|
||||
}
|
||||
```
|
||||
|
||||
enable filed value true means enable dynamic resource oversubscription, false means disable it.
|
||||
|
||||
```json
|
||||
"overSubscriptionConfig":{
|
||||
"enable": true,
|
||||
}
|
||||
```
|
||||
|
||||
enable filed value true means enable network bandwidth isolation, false means disable it.
|
||||
|
||||
```json
|
||||
"networkQosConfig":{
|
||||
"enable": true,
|
||||
}
|
||||
```
|
||||
|
||||
### CPU burst
|
||||
|
||||
Container in a pod enabled cpu burst can burst cpu quota at most equal to container's cpu limit, if many pods are using burst cpu at the same time, CPU contention will occur and affect cpu cfs scheduling. You can set pod annotation `volcano.sh/quota-burst-time` to specify custom burst quota, for example, if a container's cpu limit is 4 core, and volcano agent will set container's cgroup `cpu.cfs_quota_us` value to 400000(the basic cfs period is 100000, so 4 core cpu will be 4*100000=400000), which means container can use at most an extra 4 core cpu in a moment, if you set volcano.sh/quota-burst-time=200000, it means container can only use at most an extra 2 core cpu in a moment.
|
||||
|
||||
```yaml
|
||||
annotations:
|
||||
volcano.sh/quota-burst-time: "200000"
|
||||
```
|
||||
|
||||
### Dynamic resource oversubscription
|
||||
|
||||
The oversubscription resources computation and offline workloads eviction only take pod's resource usage into consideration by default, if you want to consider the resource utilization of the node itself, you should set flag`--include-system-usage=true` of volcano agent.
|
||||
|
||||
To avoid excessive pressure on nodes, volcano agent set an oversubscription ratio to determine the ratio of idle resource oversubscription, you can change the parameters by set flag `--oversubscription-ratio`, default value is 60, which means 60% of idle resources will be oversold, if you set `--oversubscription-ratio=100`, it means all idle resources will be oversold.
|
||||
|
||||
Volcano agent will evict offline workloads when nodes have pressure, and the eviction threshold can be configured by configMap volcano-agent-configuration, `"evictingCPUHighWatermark":80` means eviction will happed when node's cpu utilization is beyond 80% in a period of time, and current node can not schedule new pods when eviction is happening, and `"evictingCPULowWatermark":30` means node will recover schedule when node's cpu utilization is below 30%, `evictingMemoryHighWatermark` and `evictingMemoryLowWatermark` has the same meaning but for memory resource.
|
||||
|
||||
```json
|
||||
"evictingConfig":{
|
||||
"evictingCPUHighWatermark": 80,
|
||||
"evictingMemoryHighWatermark": 60,
|
||||
"evictingCPULowWatermark": 30,
|
||||
"evictingMemoryLowWatermark": 30
|
||||
}
|
||||
```
|
||||
|
||||
### Network bandwidth isolation
|
||||
|
||||
You can adjust the online and offline bandwidth watermark by modifying configMap `volcano-agent-configuration`, and `qosCheckInterval` represents the interval for monitoring bandwidth watermark by the volcano agent, please be careful to modify it.
|
||||
|
||||
```json
|
||||
"networkQosConfig":{
|
||||
"enable": true,
|
||||
"onlineBandwidthWatermarkPercent": 80,
|
||||
"offlineHighBandwidthPercent":40,
|
||||
"offlineLowBandwidthPercent": 10,
|
||||
"qosCheckInterval": 10000000
|
||||
}
|
||||
```
|
||||
Binary file not shown.
|
After Width: | Height: | Size: 2.1 MiB |
Binary file not shown.
|
After Width: | Height: | Size: 486 KiB |
43
go.mod
43
go.mod
|
|
@ -4,23 +4,29 @@ go 1.22.0
|
|||
|
||||
require (
|
||||
github.com/agiledragon/gomonkey/v2 v2.11.0
|
||||
github.com/cilium/ebpf v0.9.3
|
||||
github.com/containernetworking/cni v1.1.2
|
||||
github.com/containernetworking/plugins v1.1.1
|
||||
github.com/elastic/go-elasticsearch/v7 v7.17.7
|
||||
github.com/fsnotify/fsnotify v1.7.0
|
||||
github.com/golang/mock v1.6.0
|
||||
github.com/google/go-cmp v0.6.0
|
||||
github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510
|
||||
github.com/hashicorp/go-multierror v1.0.0
|
||||
github.com/imdario/mergo v0.3.12
|
||||
github.com/hashicorp/go-multierror v1.1.1
|
||||
github.com/imdario/mergo v0.3.16
|
||||
github.com/mitchellh/mapstructure v1.5.0
|
||||
github.com/onsi/ginkgo/v2 v2.19.0
|
||||
github.com/onsi/gomega v1.33.1
|
||||
github.com/opencontainers/runc v1.1.13
|
||||
github.com/pkg/errors v0.9.1
|
||||
github.com/prometheus/client_golang v1.19.1
|
||||
github.com/prometheus/common v0.55.0
|
||||
github.com/prometheus/prometheus v0.39.1
|
||||
github.com/spf13/cobra v1.8.1
|
||||
github.com/spf13/pflag v1.0.5
|
||||
github.com/stretchr/testify v1.9.0
|
||||
go.uber.org/automaxprocs v1.4.0
|
||||
github.com/vishvananda/netlink v1.1.1-0.20210330154013-f5de75959ad5
|
||||
go.uber.org/automaxprocs v1.5.1
|
||||
golang.org/x/crypto v0.24.0
|
||||
golang.org/x/sys v0.21.0
|
||||
golang.org/x/time v0.3.0
|
||||
|
|
@ -44,9 +50,24 @@ require (
|
|||
)
|
||||
|
||||
require (
|
||||
github.com/NYTimes/gziphandler v1.1.1 // indirect
|
||||
github.com/Microsoft/go-winio v0.6.0 // indirect
|
||||
github.com/antlr4-go/antlr/v4 v4.13.0 // indirect
|
||||
github.com/asaskevich/govalidator v0.0.0-20190424111038-f61b66f89f4a // indirect
|
||||
github.com/bits-and-blooms/bitset v1.2.0 // indirect
|
||||
github.com/cyphar/filepath-securejoin v0.2.4 // indirect
|
||||
github.com/go-task/slim-sprig/v3 v3.0.0 // indirect
|
||||
github.com/godbus/dbus/v5 v5.1.0 // indirect
|
||||
github.com/opencontainers/runtime-spec v1.0.3-0.20220909204839-494a5a6aca78 // indirect
|
||||
github.com/sirupsen/logrus v1.9.3 // indirect
|
||||
github.com/vishvananda/netns v0.0.4 // indirect
|
||||
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.53.0 // indirect
|
||||
k8s.io/cri-api v0.31.1 // indirect
|
||||
k8s.io/cri-client v0.0.0 // indirect
|
||||
k8s.io/gengo/v2 v2.0.0-20240228010128-51d4e06bde70 // indirect
|
||||
)
|
||||
|
||||
require (
|
||||
github.com/NYTimes/gziphandler v1.1.1 // indirect
|
||||
github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 // indirect
|
||||
github.com/beorn7/perks v1.0.1 // indirect
|
||||
github.com/blang/semver/v4 v4.0.0 // indirect
|
||||
github.com/cenkalti/backoff/v4 v4.3.0 // indirect
|
||||
|
|
@ -64,7 +85,6 @@ require (
|
|||
github.com/go-openapi/jsonpointer v0.19.6 // indirect
|
||||
github.com/go-openapi/jsonreference v0.20.2 // indirect
|
||||
github.com/go-openapi/swag v0.22.4 // indirect
|
||||
github.com/go-task/slim-sprig/v3 v3.0.0 // indirect
|
||||
github.com/gogo/protobuf v1.3.2 // indirect
|
||||
github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect
|
||||
github.com/golang/protobuf v1.5.4 // indirect
|
||||
|
|
@ -76,7 +96,7 @@ require (
|
|||
github.com/google/uuid v1.6.0 // indirect
|
||||
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect
|
||||
github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect
|
||||
github.com/hashicorp/errwrap v1.0.0 // indirect
|
||||
github.com/hashicorp/errwrap v1.1.0 // indirect
|
||||
github.com/inconshreveable/mousetrap v1.1.0 // indirect
|
||||
github.com/josharian/intern v1.0.0 // indirect
|
||||
github.com/json-iterator/go v1.1.12 // indirect
|
||||
|
|
@ -86,7 +106,6 @@ require (
|
|||
github.com/modern-go/reflect2 v1.0.2 // indirect
|
||||
github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect
|
||||
github.com/opencontainers/go-digest v1.0.0 // indirect
|
||||
github.com/opencontainers/runc v1.1.13 // indirect
|
||||
github.com/opencontainers/selinux v1.11.0 // indirect
|
||||
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect
|
||||
github.com/prometheus/client_model v0.6.1 // indirect
|
||||
|
|
@ -96,7 +115,6 @@ require (
|
|||
go.etcd.io/etcd/api/v3 v3.5.14 // indirect
|
||||
go.etcd.io/etcd/client/pkg/v3 v3.5.14 // indirect
|
||||
go.etcd.io/etcd/client/v3 v3.5.14 // indirect
|
||||
go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.53.0 // indirect
|
||||
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.53.0 // indirect
|
||||
go.opentelemetry.io/otel v1.28.0 // indirect
|
||||
go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.28.0 // indirect
|
||||
|
|
@ -125,8 +143,7 @@ require (
|
|||
gopkg.in/yaml.v3 v3.0.1 // indirect
|
||||
k8s.io/apiextensions-apiserver v0.25.0 // indirect
|
||||
k8s.io/cloud-provider v0.0.0 // indirect
|
||||
k8s.io/controller-manager v0.31.1 // indirect
|
||||
k8s.io/gengo/v2 v2.0.0-20240228010128-51d4e06bde70 // indirect
|
||||
k8s.io/controller-manager v0.31.1
|
||||
k8s.io/kms v0.31.1 // indirect
|
||||
k8s.io/kube-openapi v0.0.0-20240228011516-70dd3763d340 // indirect
|
||||
k8s.io/kube-scheduler v0.0.0 // indirect
|
||||
|
|
@ -138,7 +155,10 @@ require (
|
|||
)
|
||||
|
||||
replace (
|
||||
cloud.google.com/go => cloud.google.com/go v0.100.2
|
||||
github.com/opencontainers/runc => github.com/opencontainers/runc v1.0.3
|
||||
go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc => go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.19.0
|
||||
google.golang.org/grpc => google.golang.org/grpc v1.57.0
|
||||
k8s.io/api => k8s.io/api v0.31.1
|
||||
k8s.io/apiextensions-apiserver => k8s.io/apiextensions-apiserver v0.31.1
|
||||
k8s.io/apimachinery => k8s.io/apimachinery v0.31.1
|
||||
|
|
@ -152,6 +172,7 @@ replace (
|
|||
k8s.io/component-helpers => k8s.io/component-helpers v0.31.1
|
||||
k8s.io/controller-manager => k8s.io/controller-manager v0.31.1
|
||||
k8s.io/cri-api => k8s.io/cri-api v0.31.1
|
||||
k8s.io/cri-client => k8s.io/cri-client v0.31.1
|
||||
k8s.io/csi-translation-lib => k8s.io/csi-translation-lib v0.31.1
|
||||
k8s.io/dynamic-resource-allocation => k8s.io/dynamic-resource-allocation v0.31.1
|
||||
k8s.io/endpointslice => k8s.io/endpointslice v0.31.1
|
||||
|
|
|
|||
100
go.sum
100
go.sum
|
|
@ -7,10 +7,11 @@ github.com/agiledragon/gomonkey/v2 v2.11.0 h1:5oxSgA+tC1xuGsrIorR+sYiziYltmJyEZ9
|
|||
github.com/agiledragon/gomonkey/v2 v2.11.0/go.mod h1:ap1AmDzcVOAz1YpeJ3TCzIgstoaWLA6jbbgxfB4w2iY=
|
||||
github.com/antlr4-go/antlr/v4 v4.13.0 h1:lxCg3LAv+EUK6t1i0y1V6/SLeUi0eKEKdhQAlS8TVTI=
|
||||
github.com/antlr4-go/antlr/v4 v4.13.0/go.mod h1:pfChB/xh/Unjila75QW7+VU4TSnWnnk9UTnmpPaOR2g=
|
||||
github.com/asaskevich/govalidator v0.0.0-20190424111038-f61b66f89f4a h1:idn718Q4B6AGu/h5Sxe66HYVdqdGu2l9Iebqhi/AEoA=
|
||||
github.com/asaskevich/govalidator v0.0.0-20190424111038-f61b66f89f4a/go.mod h1:lB+ZfQJz7igIIfQNfa7Ml4HSf2uFQQRzpGGRXenZAgY=
|
||||
github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 h1:DklsrG3dyBCFEj5IhUbnKptjxatkF07cF2ak3yi77so=
|
||||
github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw=
|
||||
github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
|
||||
github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
|
||||
github.com/bits-and-blooms/bitset v1.2.0 h1:Kn4yilvwNtMACtf1eYDlG8H77R07mZSPbMjLyS07ChA=
|
||||
github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA=
|
||||
github.com/blang/semver/v4 v4.0.0 h1:1PFHFE6yCCTv8C1TeyNNarDzntLi7wMI5i/pzqYIsAM=
|
||||
github.com/blang/semver/v4 v4.0.0/go.mod h1:IbckMUScFkM3pff0VJDNKRiT6TG/YpiHIM2yvyW5YoQ=
|
||||
|
|
@ -19,8 +20,17 @@ github.com/cenkalti/backoff/v4 v4.3.0/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY
|
|||
github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs=
|
||||
github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
|
||||
github.com/checkpoint-restore/go-criu/v5 v5.0.0/go.mod h1:cfwC0EG7HMUenopBsUf9d89JlCLQIfgVcNsNN0t6T2M=
|
||||
github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI=
|
||||
github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI=
|
||||
github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU=
|
||||
github.com/cilium/ebpf v0.6.2/go.mod h1:4tRaxcgiL706VnOzHOdBlY8IEAIdxINsQBcU4xJJXRs=
|
||||
github.com/cilium/ebpf v0.9.3 h1:5KtxXZU+scyERvkJMEm16TbScVvuuMrlhPly78ZMbSc=
|
||||
github.com/cilium/ebpf v0.9.3/go.mod h1:w27N4UjpaQ9X/DGrSugxUG+H+NhgntDuPb5lCzxCn8A=
|
||||
github.com/containerd/console v1.0.2/go.mod h1:ytZPjGgY2oeTkAONYafi2kSj0aYggsf8acV1PGKCbzQ=
|
||||
github.com/containernetworking/cni v1.1.2 h1:wtRGZVv7olUHMOqouPpn3cXJWpJgM6+EUl31EQbXALQ=
|
||||
github.com/containernetworking/cni v1.1.2/go.mod h1:sDpYKmGVENF3s6uvMvGgldDWeG8dMxakj/u+i9ht9vw=
|
||||
github.com/containernetworking/plugins v1.1.1 h1:+AGfFigZ5TiQH00vhR8qPeSatj53eNGz0C1d3wVYlHE=
|
||||
github.com/containernetworking/plugins v1.1.1/go.mod h1:Sr5TH/eBsGLXK/h71HeLfX19sZPp3ry5uHSkI4LPxV8=
|
||||
github.com/coreos/go-semver v0.3.1 h1:yi21YpKnrx1gt5R+la8n5WgS0kCrsPp33dmEyHReZr4=
|
||||
github.com/coreos/go-semver v0.3.1/go.mod h1:irMmmIw/7yzSRPWryHsK7EYSg09caPQL03VsM8rvUec=
|
||||
github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc=
|
||||
|
|
@ -30,6 +40,8 @@ github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:ma
|
|||
github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o=
|
||||
github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
|
||||
github.com/cyphar/filepath-securejoin v0.2.2/go.mod h1:FpkQEhXnPnOthhzymB7CGsFk2G9VLXONKD9G7QGMM+4=
|
||||
github.com/cyphar/filepath-securejoin v0.2.4 h1:Ugdm7cg7i6ZK6x3xDF1oEu1nfkyfH53EtKeQYTC3kyg=
|
||||
github.com/cyphar/filepath-securejoin v0.2.4/go.mod h1:aPGpWjXOXUn2NCNjFvBE6aRxGGx79pTxQpKOJNYHHl4=
|
||||
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
||||
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
||||
github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM=
|
||||
|
|
@ -48,6 +60,10 @@ github.com/evanphx/json-patch/v5 v5.6.0/go.mod h1:G79N1coSVB93tBe7j6PhzjmR3/2Vvl
|
|||
github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg=
|
||||
github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U=
|
||||
github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k=
|
||||
github.com/frankban/quicktest v1.14.0 h1:+cqqvzZV87b4adx/5ayVOaYZ2CrvM4ejQvUdBzPPUss=
|
||||
github.com/frankban/quicktest v1.14.0/go.mod h1:NeW+ay9A/U67EYXNFA1nPE8e/tnQv/09mUdL/ijj8og=
|
||||
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
|
||||
github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ=
|
||||
github.com/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA=
|
||||
github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM=
|
||||
github.com/fxamacker/cbor/v2 v2.7.0 h1:iM5WgngdRBanHcxugY4JySA0nk1wZorNOpTgCMedv5E=
|
||||
|
|
@ -66,9 +82,12 @@ github.com/go-openapi/jsonreference v0.20.2/go.mod h1:Bl1zwGIM8/wsvqjsOQLJ/SH+En
|
|||
github.com/go-openapi/swag v0.22.3/go.mod h1:UzaqsxGiab7freDnrUUra0MwWfN/q7tE4j+VcZ0yl14=
|
||||
github.com/go-openapi/swag v0.22.4 h1:QLMzNJnMGPRNDCbySlcj1x01tzU8/9LTTL9hZZZogBU=
|
||||
github.com/go-openapi/swag v0.22.4/go.mod h1:UzaqsxGiab7freDnrUUra0MwWfN/q7tE4j+VcZ0yl14=
|
||||
github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE=
|
||||
github.com/go-task/slim-sprig/v3 v3.0.0 h1:sUs3vkvUymDpBKi3qH1YSqBQk9+9D/8M2mN1vB6EwHI=
|
||||
github.com/go-task/slim-sprig/v3 v3.0.0/go.mod h1:W848ghGpv3Qj3dhTPRyJypKRiqCdHZiAzKg9hl15HA8=
|
||||
github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA=
|
||||
github.com/godbus/dbus/v5 v5.1.0 h1:4KLkAxT3aOY8Li4FRJe/KvhoNFFxo0m6fNuFUO8QJUk=
|
||||
github.com/godbus/dbus/v5 v5.1.0/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA=
|
||||
github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q=
|
||||
github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q=
|
||||
github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg=
|
||||
|
|
@ -77,13 +96,16 @@ github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l
|
|||
github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
|
||||
github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc=
|
||||
github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs=
|
||||
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||
github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
|
||||
github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
|
||||
github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
|
||||
github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
|
||||
github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
|
||||
github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
|
||||
github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
|
||||
github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
|
||||
github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
|
||||
github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek=
|
||||
github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps=
|
||||
github.com/google/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4=
|
||||
|
|
@ -105,6 +127,7 @@ github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeN
|
|||
github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
|
||||
github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0=
|
||||
github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
|
||||
github.com/google/pprof v0.0.0-20210407192527-94a9f03dee38/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE=
|
||||
github.com/google/pprof v0.0.0-20240525223248-4bfdf5a9a2af h1:kmjWCqn2qkEml422C2Rrd27c3VGxi6a/6HNq8QmHRKM=
|
||||
github.com/google/pprof v0.0.0-20240525223248-4bfdf5a9a2af/go.mod h1:K1liHPHnj73Fdn/EKuT8nrFqBihUSKXoLYU0BuatOYo=
|
||||
github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 h1:El6M4kTTCOh6aBiKaUGG7oYTSPP8MxqL4YI3kZKwcP4=
|
||||
|
|
@ -122,12 +145,15 @@ github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4
|
|||
github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw=
|
||||
github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0=
|
||||
github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k=
|
||||
github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA=
|
||||
github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
|
||||
github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o=
|
||||
github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk=
|
||||
github.com/imdario/mergo v0.3.12 h1:b6R2BslTbIEToALKP7LxUvijTsNI9TAe80pLWN2g/HU=
|
||||
github.com/imdario/mergo v0.3.12/go.mod h1:jmQim1M+e3UYxmgPu/WyfjB3N3VflVyUjjjwH0dnCYA=
|
||||
github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I=
|
||||
github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
|
||||
github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo=
|
||||
github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM=
|
||||
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
|
||||
github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc=
|
||||
github.com/imdario/mergo v0.3.16 h1:wwQJbIsHYGMUyLSPrEq1CT16AhnhNJQ51+4fdHUnCl4=
|
||||
github.com/imdario/mergo v0.3.16/go.mod h1:WBLT9ZmE3lPoWsEzCh9LPo3TiwVN+ZKEjmz+hD27ysY=
|
||||
github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8=
|
||||
github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw=
|
||||
github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI=
|
||||
|
|
@ -142,7 +168,6 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm
|
|||
github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU=
|
||||
github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
|
||||
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
|
||||
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
|
||||
github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
|
||||
github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE=
|
||||
github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk=
|
||||
|
|
@ -167,12 +192,20 @@ github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq
|
|||
github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ=
|
||||
github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU=
|
||||
github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
|
||||
github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A=
|
||||
github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE=
|
||||
github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU=
|
||||
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
|
||||
github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk=
|
||||
github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0=
|
||||
github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE=
|
||||
github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU=
|
||||
github.com/onsi/ginkgo/v2 v2.1.3/go.mod h1:vw5CSIxN1JObi/U8gcbwft7ZxR2dgaR70JSE3/PpL4c=
|
||||
github.com/onsi/ginkgo/v2 v2.19.0 h1:9Cnnf7UHo57Hy3k6/m5k3dRfGTMXGvxhHFvkDTCTpvA=
|
||||
github.com/onsi/ginkgo/v2 v2.19.0/go.mod h1:rlwLi9PilAFJ8jCg9UE1QP6VBpd6/xj3SRC0d6TU0To=
|
||||
github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY=
|
||||
github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo=
|
||||
github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY=
|
||||
github.com/onsi/gomega v1.33.1 h1:dsYjIxxSR755MDmKVsaFQTE22ChNBcuuTWgkUDSubOk=
|
||||
github.com/onsi/gomega v1.33.1/go.mod h1:U4R44UsT+9eLIaYRB2a5qajjtQYn0hauxvRm16AVYg0=
|
||||
github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U=
|
||||
|
|
@ -180,6 +213,8 @@ github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3I
|
|||
github.com/opencontainers/runc v1.0.3 h1:1hbqejyQWCJBvtKAfdO0b1FmaEf2z/bxnjqbARass5k=
|
||||
github.com/opencontainers/runc v1.0.3/go.mod h1:aTaHFFwQXuA71CiyxOdFFIorAoemI04suvGRQFzWTD0=
|
||||
github.com/opencontainers/runtime-spec v1.0.3-0.20210326190908-1c3f411f0417/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0=
|
||||
github.com/opencontainers/runtime-spec v1.0.3-0.20220909204839-494a5a6aca78 h1:R5M2qXZiK/mWPMT4VldCOiSL9HIAMuxQZWdG0CSM5+4=
|
||||
github.com/opencontainers/runtime-spec v1.0.3-0.20220909204839-494a5a6aca78/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0=
|
||||
github.com/opencontainers/selinux v1.8.2/go.mod h1:MUIHuUEvKB1wtJjQdOyYRgOnLD2xAPP8dBsCoU0KuF8=
|
||||
github.com/opencontainers/selinux v1.11.0 h1:+5Zbo97w3Lbmb3PeqQtpmTkMwsW5nRI3YaLpt7tQ7oU=
|
||||
github.com/opencontainers/selinux v1.11.0/go.mod h1:E5dMC3VPuVvVHDYmi78qvhJp8+M586T4DlDRYpFkyec=
|
||||
|
|
@ -189,6 +224,8 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE
|
|||
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
|
||||
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U=
|
||||
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
|
||||
github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g=
|
||||
github.com/prashantv/gostub v1.1.0/go.mod h1:A5zLQHz7ieHGG7is6LLXLz7I8+3LZzsrV0P1IAHhP5U=
|
||||
github.com/prometheus/client_golang v1.19.1 h1:wZWJDwK+NameRJuPGDhlnFgx8e8HN3XHQeLaYJFJBOE=
|
||||
github.com/prometheus/client_golang v1.19.1/go.mod h1:mP78NwGzrVks5S2H6ab8+ZZGJLZUq1hoULYBAYBw1Ho=
|
||||
github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E=
|
||||
|
|
@ -197,11 +234,15 @@ github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G
|
|||
github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8=
|
||||
github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc=
|
||||
github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk=
|
||||
github.com/prometheus/prometheus v0.39.1 h1:abZM6A+sKAv2eKTbRIaHq4amM/nT07MuxRm0+QTaTj0=
|
||||
github.com/prometheus/prometheus v0.39.1/go.mod h1:GjQjgLhHMc0oo4Ko7qt/yBSJMY4hUoiAZwsYQgjaePA=
|
||||
github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8=
|
||||
github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4=
|
||||
github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
|
||||
github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
|
||||
github.com/seccomp/libseccomp-golang v0.9.1/go.mod h1:GbW5+tmTXfcxTToHLXlScSlAvWlF4P2Ca7zGrPiEpWo=
|
||||
github.com/seccomp/libseccomp-golang v0.10.0 h1:aA4bp+/Zzi0BnWZ2F1wgNBs5gTpm+na2rWM6M9YjLpY=
|
||||
github.com/seccomp/libseccomp-golang v0.10.0/go.mod h1:JA8cRccbGaA1s33RQf7Y1+q9gHmZX1yB/z9WDN1C6fg=
|
||||
github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc=
|
||||
github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0=
|
||||
github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ=
|
||||
|
|
@ -221,8 +262,8 @@ github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSS
|
|||
github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=
|
||||
github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
|
||||
github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
|
||||
github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
|
||||
github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA=
|
||||
github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
|
||||
github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
|
||||
github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
|
||||
github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4=
|
||||
|
|
@ -233,7 +274,12 @@ github.com/tmc/grpc-websocket-proxy v0.0.0-20220101234140-673ab2c3ae75 h1:6fotK7
|
|||
github.com/tmc/grpc-websocket-proxy v0.0.0-20220101234140-673ab2c3ae75/go.mod h1:KO6IkyS8Y3j8OdNO85qEYBsRPuteD+YciPomcXdrMnk=
|
||||
github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
|
||||
github.com/vishvananda/netlink v1.1.0/go.mod h1:cTgwzPIzzgDAYoQrMm0EdrjRUBkTqKYppBueQtXaqoE=
|
||||
github.com/vishvananda/netlink v1.1.1-0.20210330154013-f5de75959ad5 h1:+UB2BJA852UkGH42H+Oee69djmxS3ANzl2b/JtT1YiA=
|
||||
github.com/vishvananda/netlink v1.1.1-0.20210330154013-f5de75959ad5/go.mod h1:twkDnbuQxJYemMlGd4JFIcuhgX83tXhKS2B/PRMpOho=
|
||||
github.com/vishvananda/netns v0.0.0-20191106174202-0a2b9b5464df/go.mod h1:JP3t17pCcGlemwknint6hfoeCVQrEMVwxRLRjXpq+BU=
|
||||
github.com/vishvananda/netns v0.0.0-20200728191858-db3c7e526aae/go.mod h1:DD4vA1DwXk04H54A1oHXtwZmA0grkVMdPxx/VGLCah0=
|
||||
github.com/vishvananda/netns v0.0.4 h1:Oeaw1EM2JMxD51g9uhtC0D7erkIjgmj8+JZc26m1YX8=
|
||||
github.com/vishvananda/netns v0.0.4/go.mod h1:SpkAiCQRtJ6TvvxPnOSyH3BMl6unz3xZlaprSwhNNJM=
|
||||
github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM=
|
||||
github.com/x448/float16 v0.8.4/go.mod h1:14CWIYCyZA/cWjXOioeEpHeN/83MdbZDRQHoFcYsOfg=
|
||||
github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8=
|
||||
|
|
@ -265,8 +311,8 @@ go.opentelemetry.io/otel v1.28.0 h1:/SqNcYk+idO0CxKEUOtKQClMK/MimZihKYMruSMViUo=
|
|||
go.opentelemetry.io/otel v1.28.0/go.mod h1:q68ijF8Fc8CnMHKyzqL6akLO46ePnjkgfIMIjUIX9z4=
|
||||
go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.28.0 h1:3Q/xZUyC1BBkualc9ROb4G8qkH90LXEIICcs5zv1OYY=
|
||||
go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.28.0/go.mod h1:s75jGIWA9OfCMzF0xr+ZgfrB5FEbbV7UuYo32ahUiFI=
|
||||
go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.27.0 h1:qFffATk0X+HD+f1Z8lswGiOQYKHRlzfmdJm0wEaVrFA=
|
||||
go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.27.0/go.mod h1:MOiCmryaYtc+V0Ei+Tx9o5S1ZjA7kzLucuVuyzBZloQ=
|
||||
go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.19.0 h1:3d+S281UTjM+AbF31XSOYn1qXn3BgIdWl8HNEpx08Jk=
|
||||
go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.19.0/go.mod h1:0+KuTDyKL4gjKCF75pHOX4wuzYDUZYfAQdSu43o+Z2I=
|
||||
go.opentelemetry.io/otel/metric v1.28.0 h1:f0HGvSl1KRAU1DLgLGFjrwVyismPlnuU6JD6bOeuA5Q=
|
||||
go.opentelemetry.io/otel/metric v1.28.0/go.mod h1:Fb1eVBFZmLVTMb6PPohq3TO9IIhUisDsbJoL/+uQW4s=
|
||||
go.opentelemetry.io/otel/sdk v1.28.0 h1:b9d7hIry8yZsgtbmM0DKyPWMMUMlK9NEKuIG4aBqWyE=
|
||||
|
|
@ -275,8 +321,8 @@ go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+
|
|||
go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI=
|
||||
go.opentelemetry.io/proto/otlp v1.3.1 h1:TrMUixzpM0yuc/znrFTP9MMRh8trP93mkCiDVeXrui0=
|
||||
go.opentelemetry.io/proto/otlp v1.3.1/go.mod h1:0X1WI4de4ZsLrrJNLAQbFeLCm3T7yBkR0XqQ7niQU+8=
|
||||
go.uber.org/automaxprocs v1.4.0 h1:CpDZl6aOlLhReez+8S3eEotD7Jx0Os++lemPlMULQP0=
|
||||
go.uber.org/automaxprocs v1.4.0/go.mod h1:/mTEdr7LvHhs0v7mjdxDreTz1OG5zdZGqgOnhWiR/+Q=
|
||||
go.uber.org/automaxprocs v1.5.1 h1:e1YG66Lrk73dn4qhg8WFSvhF0JuFQF0ERIp4rpuV8Qk=
|
||||
go.uber.org/automaxprocs v1.5.1/go.mod h1:BF4eumQw0P9GtnuxxovUd06vwm1o18oMzFtK66vU6XU=
|
||||
go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto=
|
||||
go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE=
|
||||
go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0=
|
||||
|
|
@ -295,36 +341,51 @@ golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
|
|||
golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
|
||||
golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA=
|
||||
golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c=
|
||||
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||
golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||
golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||
golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||
golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
|
||||
golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
|
||||
golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
|
||||
golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM=
|
||||
golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk=
|
||||
golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ=
|
||||
golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE=
|
||||
golang.org/x/oauth2 v0.21.0 h1:tsimM75w1tF/uws5rbeHzIWxEqElMehnc+iW793zsZs=
|
||||
golang.org/x/oauth2 v0.21.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI=
|
||||
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M=
|
||||
golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk=
|
||||
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20190606203320-7fc4e5ec1444/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20191115151921-52ab43148777/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20200217220822-9197077df867/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20200728102440-3e129f6d46b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20200909081042-eff7692f9009/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20210426230700-d19ff857e887/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.21.0 h1:rF+pYz3DAGSQAxAu1CbC7catZg4ebC4UIeIhKxBZvws=
|
||||
golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA=
|
||||
golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
|
||||
|
|
@ -332,6 +393,7 @@ golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA=
|
|||
golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0=
|
||||
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||
golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
|
||||
golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
|
||||
golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4=
|
||||
golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI=
|
||||
golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4=
|
||||
|
|
@ -340,6 +402,7 @@ golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGm
|
|||
golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||
golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
|
||||
golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
|
||||
golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
|
||||
golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
|
||||
golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
|
||||
golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d h1:vU5i/LfpvrRCpgM/VPfJLg5KjxD3E+hfT1SH+d9zLwg=
|
||||
|
|
@ -356,8 +419,8 @@ google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157 h1:
|
|||
google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU=
|
||||
google.golang.org/genproto/googleapis/rpc v0.0.0-20240701130421-f6361c86f094 h1:BwIjyKYGsK9dMCBOorzRri8MQwmi7mT9rGHsCEinZkA=
|
||||
google.golang.org/genproto/googleapis/rpc v0.0.0-20240701130421-f6361c86f094/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY=
|
||||
google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc=
|
||||
google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ=
|
||||
google.golang.org/grpc v1.57.0 h1:kfzNeI/klCGD2YPMUlaGNT3pxvYfga7smW3Vth8Zsiw=
|
||||
google.golang.org/grpc v1.57.0/go.mod h1:Sd+9RMTACXwmub0zcNY2c4arhtrbBYD1AUHI/dt16Mo=
|
||||
google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
|
||||
google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
|
||||
google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
|
||||
|
|
@ -369,11 +432,11 @@ google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQ
|
|||
google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg=
|
||||
google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw=
|
||||
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
|
||||
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
|
||||
gopkg.in/evanphx/json-patch.v4 v4.12.0 h1:n6jtcsulIzXPJaxegRbvFNNrZDjbij7ny3gmSPG+6V4=
|
||||
gopkg.in/evanphx/json-patch.v4 v4.12.0/go.mod h1:p8EYWUEYMpynmqDbY58zCKCFZw8pRWMG4EsWvDvM72M=
|
||||
gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
|
||||
gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc=
|
||||
gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw=
|
||||
gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc=
|
||||
|
|
@ -381,6 +444,7 @@ gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYs
|
|||
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=
|
||||
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
|
||||
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||
gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||
gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||
gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||
gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
|
||||
|
|
@ -408,6 +472,10 @@ k8s.io/component-helpers v0.31.1 h1:5hZUf3747atdgtR3gPntrG35rC2CkK7rYq2KUraz6Os=
|
|||
k8s.io/component-helpers v0.31.1/go.mod h1:ye0Gi8KzFNTfpIuzvVDtxJQMP/0Owkukf1vGf22Hl6U=
|
||||
k8s.io/controller-manager v0.31.1 h1:bwiy8y//EG5lJL2mdbOvZWrOgw2EXXIvwp95VYgoIis=
|
||||
k8s.io/controller-manager v0.31.1/go.mod h1:O440MSE6EI1AEVhB2Fc8FYqv6r8BHrSXjm5aj3886No=
|
||||
k8s.io/cri-api v0.31.1 h1:x0aI8yTI7Ho4c8tpuig8NwI/MRe+VhjiYyyebC2xphQ=
|
||||
k8s.io/cri-api v0.31.1/go.mod h1:Po3TMAYH/+KrZabi7QiwQI4a692oZcUOUThd/rqwxrI=
|
||||
k8s.io/cri-client v0.31.1 h1:w5D7BAhiaSVVDZqHs7YUZPpuUCybx8tCxfdBuDBw7zo=
|
||||
k8s.io/cri-client v0.31.1/go.mod h1:voVfZexZQwvlf/JD8w30sGN0k22LRcHRfCj7+m4kAXE=
|
||||
k8s.io/csi-translation-lib v0.31.1 h1:ps9kya8+ih0CVL59JO2B4AYH8U/e3WLQxl9sx19NjjM=
|
||||
k8s.io/csi-translation-lib v0.31.1/go.mod h1:VeYSucPZJbAt6RT25AzfG7WjyxCcmqxtr4V/CaDdNZc=
|
||||
k8s.io/gengo/v2 v2.0.0-20240228010128-51d4e06bde70 h1:NGrVE502P0s0/1hudf8zjgwki1X/TByhmAoILTarmzo=
|
||||
|
|
|
|||
|
|
@ -31,3 +31,14 @@ if ! diff ${VK_ROOT}/installer/volcano-development.yaml ${RELEASE_FOLDER}/volcan
|
|||
} >&2
|
||||
false
|
||||
fi
|
||||
|
||||
if ! diff ${VK_ROOT}/installer/volcano-agent-development.yaml ${RELEASE_FOLDER}/volcano-agent-latest.yaml ; then
|
||||
{
|
||||
echo
|
||||
echo "The Generated yaml is different from the one in installer/volcano-agent-development.yaml"
|
||||
echo "please run 'make generate-yaml TAG=latest RELEASE_DIR=installer \
|
||||
&& mv ${VK_ROOT}/installer/volcano-agent-latest.yaml ${VK_ROOT}/installer/volcano-agent-development.yaml' to update"
|
||||
echo
|
||||
} >&2
|
||||
false
|
||||
fi
|
||||
|
|
@ -28,6 +28,8 @@ export HELM_VER=${HELM_VER:-v3.6.3}
|
|||
export VOLCANO_IMAGE_TAG=${TAG:-"latest"}
|
||||
export YAML_FILENAME=volcano-${VOLCANO_IMAGE_TAG}.yaml
|
||||
export MONITOR_YAML_FILENAME=volcano-monitoring-${VOLCANO_IMAGE_TAG}.yaml
|
||||
export AGENT_YAML_FILENAME=volcano-agent-${VOLCANO_IMAGE_TAG}.yaml
|
||||
|
||||
export CRD_VERSION=${CRD_VERSION:-v1}
|
||||
|
||||
case $CRD_VERSION in
|
||||
|
|
@ -103,6 +105,8 @@ fi
|
|||
|
||||
DEPLOYMENT_FILE=${RELEASE_FOLDER}/${YAML_FILENAME}
|
||||
MONITOR_DEPLOYMENT_YAML_FILENAME=${RELEASE_FOLDER}/${MONITOR_YAML_FILENAME}
|
||||
AGENT_DEPLOYMENT_YAML_FILENAME=${RELEASE_FOLDER}/${AGENT_YAML_FILENAME}
|
||||
|
||||
echo "Generating volcano yaml file into ${DEPLOYMENT_FILE}"
|
||||
|
||||
if [[ -f ${DEPLOYMENT_FILE} ]];then
|
||||
|
|
@ -113,6 +117,10 @@ if [[ -f ${MONITOR_DEPLOYMENT_YAML_FILENAME} ]];then
|
|||
rm ${MONITOR_DEPLOYMENT_YAML_FILENAME}
|
||||
fi
|
||||
|
||||
if [[ -f ${AGENT_DEPLOYMENT_YAML_FILENAME} ]];then
|
||||
rm "${AGENT_DEPLOYMENT_YAML_FILENAME}"
|
||||
fi
|
||||
|
||||
# Namespace
|
||||
cat ${VK_ROOT}/installer/namespace.yaml > ${DEPLOYMENT_FILE}
|
||||
|
||||
|
|
@ -145,3 +153,9 @@ ${HELM_BIN_DIR}/helm template ${VK_ROOT}/installer/helm/chart/volcano --namespac
|
|||
-s templates/kubestatemetrics.yaml \
|
||||
-s templates/grafana.yaml \
|
||||
>> ${MONITOR_DEPLOYMENT_YAML_FILENAME}
|
||||
|
||||
# Agent
|
||||
${HELM_BIN_DIR}/helm template ${VK_ROOT}/installer/helm/chart/volcano --namespace volcano-system \
|
||||
--name-template volcano --set basic.image_tag_version=${VOLCANO_IMAGE_TAG} --set custom.colocation_enable=true \
|
||||
-s templates/agent.yaml \
|
||||
>> ${AGENT_DEPLOYMENT_YAML_FILENAME}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,57 @@
|
|||
#!/bin/sh
|
||||
|
||||
# Copyright 2024 The Volcano Authors.
|
||||
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
VOLCANO_AGENT_LOG_DIR="/var/log/volcano/agent"
|
||||
VOLCANO_AGENT_LOG_PATH="${VOLCANO_AGENT_LOG_DIR}/volcano-agent.log"
|
||||
NETWORK_QOS_LOG_PATH="${VOLCANO_AGENT_LOG_DIR}/network-qos.log"
|
||||
NETWORK_QOS_TOOLS_LOG_PATH="${VOLCANO_AGENT_LOG_DIR}/network-qos-tools.log"
|
||||
|
||||
MEMORY_QOS_ENABLED_PATH="/host/proc/sys/vm/memcg_qos_enable"
|
||||
LOAD_BALANCE_ENABLED_PATH="/host/proc/sys/kernel/sched_prio_load_balance_enabled"
|
||||
|
||||
function log()
|
||||
{
|
||||
echo "[`date "+%Y-%m-%d %H:%M:%S"`] $*"
|
||||
}
|
||||
|
||||
function set_sched_prio_load_balance_enabled() {
|
||||
log "Start to enable cpu load balance"
|
||||
if [[ ! -f ${LOAD_BALANCE_ENABLED_PATH} ]]; then
|
||||
log "Enable cpu load balance failed, file(${LOAD_BALANCE_ENABLED_PATH}) does not existed"
|
||||
return 0
|
||||
fi
|
||||
cat ${LOAD_BALANCE_ENABLED_PATH}
|
||||
echo 1 > ${LOAD_BALANCE_ENABLED_PATH}
|
||||
log "Successfully enabled cpu load balance"
|
||||
}
|
||||
|
||||
function set_memory_qos_enabled(){
|
||||
log "Start to enable memory qos enable"
|
||||
if [[ ! -f ${MEMORY_QOS_ENABLED_PATH} ]]; then
|
||||
log "Enable memory cgroup qos failed, file(${MEMORY_QOS_ENABLED_PATH}) does not existed"
|
||||
return 0
|
||||
fi
|
||||
cat ${MEMORY_QOS_ENABLED_PATH}
|
||||
echo 1 > ${MEMORY_QOS_ENABLED_PATH}
|
||||
log "Successfully enabled memory qos"
|
||||
}
|
||||
|
||||
touch ${VOLCANO_AGENT_LOG_PATH}
|
||||
touch ${NETWORK_QOS_LOG_PATH}
|
||||
touch ${NETWORK_QOS_TOOLS_LOG_PATH}
|
||||
|
||||
set_memory_qos_enabled
|
||||
set_sched_prio_load_balance_enabled
|
||||
|
|
@ -0,0 +1,38 @@
|
|||
# Copyright 2024 The Volcano Authors.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
ARG OPEN_EULER_IMAGE_TAG
|
||||
ARG BWM_RPM_NAME
|
||||
|
||||
FROM golang:1.22.2 AS builder
|
||||
WORKDIR /go/src/volcano.sh/
|
||||
COPY go.mod go.sum ./
|
||||
RUN go mod download
|
||||
ADD . volcano
|
||||
RUN cd volcano && make vc-agent
|
||||
|
||||
FROM openeuler/openeuler:${OPEN_EULER_IMAGE_TAG} AS repo
|
||||
WORKDIR /
|
||||
RUN yum install -y cpio && \
|
||||
yum install -y --downloadonly --destdir=./ oncn-bwm && \
|
||||
rpm2cpio $(ls | grep oncn-bwm) | cpio -div
|
||||
|
||||
FROM alpine:latest
|
||||
RUN apk add sudo
|
||||
COPY --from=builder /go/src/volcano.sh/volcano/_output/bin/vc-agent /vc-agent
|
||||
COPY --from=builder /go/src/volcano.sh/volcano/_output/bin/network-qos \
|
||||
/go/src/volcano.sh/volcano/installer/build/volcano-agent/install.sh /usr/local/bin/
|
||||
COPY --from=repo /usr/share/bwmcli/bwm_tc.o /usr/local/bin/
|
||||
RUN chmod +x /usr/local/bin/install.sh
|
||||
|
||||
|
|
@ -0,0 +1,240 @@
|
|||
{{- if .Values.custom.colocation_enable }}
|
||||
{{ $agent_affinity := or .Values.custom.agent_affinity .Values.custom.default_affinity }}
|
||||
{{ $agent_tolerations := or .Values.custom.agent_tolerations .Values.custom.default_tolerations }}
|
||||
{{ $agent_sc := or .Values.custom.agent_sc .Values.custom.default_sc }}
|
||||
{{ $agent_ns := or .Values.custom.agent_ns .Values.custom.default_ns }}
|
||||
apiVersion: apps/v1
|
||||
kind: DaemonSet
|
||||
metadata:
|
||||
name: {{ .Release.Name }}-agent
|
||||
namespace: {{ .Release.Namespace }}
|
||||
spec:
|
||||
selector:
|
||||
matchLabels:
|
||||
name: volcano-agent
|
||||
template:
|
||||
metadata:
|
||||
name: volcano-agent
|
||||
labels:
|
||||
name: volcano-agent
|
||||
annotations:
|
||||
prometheus.io/path: /metrics
|
||||
prometheus.io/port: '3300'
|
||||
prometheus.io/scheme: http
|
||||
prometheus.io/scrape: 'true'
|
||||
spec:
|
||||
{{- if $agent_tolerations }}
|
||||
tolerations: {{- toYaml $agent_tolerations | nindent 8 }}
|
||||
{{- end }}
|
||||
{{- if $agent_ns }}
|
||||
nodeSelector: {{- toYaml $agent_ns | nindent 8 }}
|
||||
{{- end }}
|
||||
{{- if $agent_affinity }}
|
||||
affinity:
|
||||
{{- toYaml $agent_affinity | nindent 8 }}
|
||||
{{- end }}
|
||||
{{- if $agent_sc }}
|
||||
securityContext:
|
||||
{{- toYaml $agent_sc | nindent 8 }}
|
||||
{{- end }}
|
||||
{{- if .Values.basic.image_pull_secret }}
|
||||
imagePullSecrets:
|
||||
- name: {{ .Values.basic.image_pull_secret }}
|
||||
{{- end }}
|
||||
serviceAccountName: {{ .Release.Name }}-agent
|
||||
hostNetwork: true
|
||||
priorityClassName: system-node-critical
|
||||
restartPolicy: Always
|
||||
dnsPolicy: Default
|
||||
volumes:
|
||||
- name: bwm-dir
|
||||
hostPath:
|
||||
path: /usr/share/bwmcli/
|
||||
type: DirectoryOrCreate
|
||||
- name: cni-plugin-dir
|
||||
hostPath:
|
||||
path: /opt/cni/bin
|
||||
type: Directory
|
||||
- name: host-etc
|
||||
hostPath:
|
||||
path: /etc
|
||||
type: Directory
|
||||
- name: host-sys-fs
|
||||
hostPath:
|
||||
path: /sys/fs
|
||||
type: Directory
|
||||
- name: host-proc-sys
|
||||
hostPath:
|
||||
path: /proc/sys
|
||||
type: Directory
|
||||
- name: log
|
||||
hostPath:
|
||||
path: /var/log/volcano/agent
|
||||
type: ''
|
||||
- name: localtime
|
||||
hostPath:
|
||||
path: /etc/localtime
|
||||
type: ''
|
||||
- name: kubelet-cpu-manager-policy
|
||||
hostPath:
|
||||
path: /var/lib/kubelet/
|
||||
type: ''
|
||||
- name: proc-stat
|
||||
hostPath:
|
||||
path: /proc/stat
|
||||
type: File
|
||||
initContainers:
|
||||
- name: volcano-agent-init
|
||||
image: {{ .Values.basic.image_registry }}/{{.Values.basic.agent_image_name}}:{{.Values.basic.image_tag_version}}
|
||||
command:
|
||||
- /bin/sh
|
||||
- '-c'
|
||||
- /usr/local/bin/install.sh
|
||||
{{- if .Values.custom.agent_resources }}
|
||||
resources:
|
||||
{{- toYaml .Values.custom.agent_resources | nindent 12 }}
|
||||
{{- end }}
|
||||
volumeMounts:
|
||||
- name: bwm-dir
|
||||
mountPath: /usr/share/bwmcli
|
||||
- name: cni-plugin-dir
|
||||
mountPath: /opt/cni/bin
|
||||
- name: host-etc
|
||||
mountPath: /host/etc
|
||||
- mountPath: /var/log/volcano/agent
|
||||
name: log
|
||||
- name: host-proc-sys
|
||||
mountPath: /host/proc/sys
|
||||
terminationMessagePath: /dev/termination-log
|
||||
terminationMessagePolicy: File
|
||||
imagePullPolicy: {{ .Values.basic.image_pull_policy }}
|
||||
containers:
|
||||
- name: volcano-agent
|
||||
image: {{ .Values.basic.image_registry }}/{{.Values.basic.agent_image_name}}:{{.Values.basic.image_tag_version}}
|
||||
command:
|
||||
- /bin/sh
|
||||
- '-c'
|
||||
- /vc-agent --v=2 1>>/var/log/volcano/agent/volcano-agent.log 2>&1
|
||||
env:
|
||||
- name: SYS_FS_PATH
|
||||
value: /host/sys/fs
|
||||
- name: CNI_CONF_FILE_PATH
|
||||
value: /host{{ .Values.custom.agent_cni_config_path }}
|
||||
- name: KUBE_NODE_NAME
|
||||
valueFrom:
|
||||
fieldRef:
|
||||
apiVersion: v1
|
||||
fieldPath: spec.nodeName
|
||||
- name: KUBE_POD_NAME
|
||||
valueFrom:
|
||||
fieldRef:
|
||||
apiVersion: v1
|
||||
fieldPath: metadata.name
|
||||
- name: KUBE_POD_NAMESPACE
|
||||
valueFrom:
|
||||
fieldRef:
|
||||
apiVersion: v1
|
||||
fieldPath: metadata.namespace
|
||||
{{- if .Values.custom.agent_resources }}
|
||||
resources:
|
||||
{{- toYaml .Values.custom.agent_resources | nindent 12 }}
|
||||
{{- end }}
|
||||
volumeMounts:
|
||||
- name: bwm-dir
|
||||
mountPath: /usr/share/bwmcli
|
||||
- name: cni-plugin-dir
|
||||
mountPath: /opt/cni/bin
|
||||
- name: host-etc
|
||||
mountPath: /host/etc
|
||||
- mountPath: /var/log/volcano/agent
|
||||
name: log
|
||||
- name: host-sys-fs
|
||||
mountPath: /host/sys/fs
|
||||
mountPropagation: HostToContainer
|
||||
- name: host-proc-sys
|
||||
mountPath: /host/proc/sys
|
||||
- name: localtime
|
||||
readOnly: true
|
||||
mountPath: /etc/localtime
|
||||
- name: kubelet-cpu-manager-policy
|
||||
readOnly: true
|
||||
mountPath: /var/lib/kubelet
|
||||
- name: proc-stat
|
||||
readOnly: true
|
||||
mountPath: /host/proc/stat
|
||||
livenessProbe:
|
||||
httpGet:
|
||||
path: /healthz
|
||||
port: 3300
|
||||
scheme: HTTP
|
||||
initialDelaySeconds: 60
|
||||
timeoutSeconds: 5
|
||||
periodSeconds: 10
|
||||
successThreshold: 1
|
||||
failureThreshold: 5
|
||||
imagePullPolicy: {{ .Values.basic.image_pull_policy }}
|
||||
updateStrategy:
|
||||
type: RollingUpdate
|
||||
rollingUpdate:
|
||||
maxUnavailable: 10%
|
||||
maxSurge: 0
|
||||
revisionHistoryLimit: 10
|
||||
|
||||
---
|
||||
apiVersion: v1
|
||||
kind: ServiceAccount
|
||||
metadata:
|
||||
labels:
|
||||
app: volcano-agent
|
||||
name: {{ .Release.Name }}-agent
|
||||
namespace: {{ .Release.Namespace }}
|
||||
|
||||
---
|
||||
kind: ClusterRole
|
||||
apiVersion: rbac.authorization.k8s.io/v1
|
||||
metadata:
|
||||
labels:
|
||||
app: volcano-agent
|
||||
name: {{ .Release.Name }}-agent
|
||||
rules:
|
||||
- apiGroups: [""]
|
||||
resources: [ "nodes", "nodes/status" ]
|
||||
verbs: [ "get", "list", "watch", "update", "patch" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "secrets" ]
|
||||
verbs: [ "get", "list", "watch" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "configmaps" ]
|
||||
verbs: [ "get", "list", "watch", "create", "update" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "pods" ]
|
||||
verbs: [ "get", "list", "watch" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "pods/eviction" ]
|
||||
verbs: [ "create" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "configmaps" ]
|
||||
verbs: ["get", "list", "watch" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "events" ]
|
||||
verbs: [ "list", "watch", "create", "update", "patch" ]
|
||||
|
||||
---
|
||||
kind: ClusterRoleBinding
|
||||
apiVersion: rbac.authorization.k8s.io/v1
|
||||
metadata:
|
||||
labels:
|
||||
app: volcano-agent
|
||||
name: {{ .Release.Name }}-agent-role
|
||||
subjects:
|
||||
- kind: ServiceAccount
|
||||
name: {{ .Release.Name }}-agent
|
||||
namespace: {{ .Release.Namespace }}
|
||||
- kind: User
|
||||
apiGroup: rbac.authorization.k8s.io
|
||||
name: {{ .Release.Name }}-agent
|
||||
roleRef:
|
||||
kind: ClusterRole
|
||||
name: {{ .Release.Name }}-agent
|
||||
apiGroup: rbac.authorization.k8s.io
|
||||
{{- end }}
|
||||
|
|
@ -2,6 +2,7 @@ basic:
|
|||
controller_image_name: "volcanosh/vc-controller-manager"
|
||||
scheduler_image_name: "volcanosh/vc-scheduler"
|
||||
admission_image_name: "volcanosh/vc-webhook-manager"
|
||||
agent_image_name: "volcanosh/vc-agent"
|
||||
admission_secret_name: "volcano-admission-secret"
|
||||
admission_config_file: "config/volcano-admission.conf"
|
||||
scheduler_config_file: "config/volcano-scheduler.conf"
|
||||
|
|
@ -23,6 +24,7 @@ custom:
|
|||
scheduler_name: ~
|
||||
leader_elect_enable: false
|
||||
enabled_admissions: "/jobs/mutate,/jobs/validate,/podgroups/mutate,/pods/validate,/pods/mutate,/queues/mutate,/queues/validate"
|
||||
colocation_enable: false
|
||||
|
||||
# Override the configuration for admission or scheduler.
|
||||
# For example:
|
||||
|
|
@ -62,6 +64,7 @@ custom:
|
|||
admission_affinity: ~
|
||||
controller_affinity: ~
|
||||
scheduler_affinity: ~
|
||||
agent_affinity: ~
|
||||
|
||||
# Specify tolerations for all main Volcano components or per component
|
||||
# For example:
|
||||
|
|
@ -74,6 +77,10 @@ custom:
|
|||
admission_tolerations: ~
|
||||
controller_tolerations: ~
|
||||
scheduler_tolerations: ~
|
||||
agent_tolerations:
|
||||
- key: "volcano.sh/offline-job-evicting"
|
||||
operator: "Exists"
|
||||
effect: "NoSchedule"
|
||||
|
||||
# Specify securityContext for all main Volcano components or per component
|
||||
# For example:
|
||||
|
|
@ -85,6 +92,7 @@ custom:
|
|||
scheduler_sc: ~
|
||||
admission_sc: ~
|
||||
controller_sc: ~
|
||||
agent_sc: ~
|
||||
|
||||
# Specify nodeSelector for all main Volcano components or per component
|
||||
# For example:
|
||||
|
|
@ -96,6 +104,7 @@ custom:
|
|||
scheduler_ns: ~
|
||||
controller_ns: ~
|
||||
kube_state_metrics_ns: ~
|
||||
agent_ns: ~
|
||||
|
||||
# Specify labels for Volcano main component deployments and pods
|
||||
# For example:
|
||||
|
|
@ -123,6 +132,7 @@ custom:
|
|||
admission_resources: ~
|
||||
scheduler_resources: ~
|
||||
controller_resources: ~
|
||||
agent_resources: ~
|
||||
|
||||
|
||||
# Specify additional namespace selector expressions for Volcano admission webhooks.
|
||||
|
|
@ -161,6 +171,9 @@ custom:
|
|||
scheduler_default_csc: ~
|
||||
controller_default_csc: ~
|
||||
|
||||
# Specify agent cni config path.
|
||||
agent_cni_config_path: /etc/cni/net.d/cni.conflist
|
||||
|
||||
service:
|
||||
# @param service.ipFamilyPolicy [string], support SingleStack, PreferDualStack and RequireDualStack
|
||||
#
|
||||
|
|
@ -171,4 +184,6 @@ service:
|
|||
# ipFamilies:
|
||||
# - IPv6
|
||||
# - IPv4
|
||||
ipFamilies: []
|
||||
ipFamilies: []
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,214 @@
|
|||
---
|
||||
# Source: volcano/templates/agent.yaml
|
||||
apiVersion: v1
|
||||
kind: ServiceAccount
|
||||
metadata:
|
||||
labels:
|
||||
app: volcano-agent
|
||||
name: volcano-agent
|
||||
namespace: volcano-system
|
||||
---
|
||||
# Source: volcano/templates/agent.yaml
|
||||
kind: ClusterRole
|
||||
apiVersion: rbac.authorization.k8s.io/v1
|
||||
metadata:
|
||||
labels:
|
||||
app: volcano-agent
|
||||
name: volcano-agent
|
||||
rules:
|
||||
- apiGroups: [""]
|
||||
resources: [ "nodes", "nodes/status" ]
|
||||
verbs: [ "get", "list", "watch", "update", "patch" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "secrets" ]
|
||||
verbs: [ "get", "list", "watch" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "configmaps" ]
|
||||
verbs: [ "get", "list", "watch", "create", "update" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "pods" ]
|
||||
verbs: [ "get", "list", "watch" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "pods/eviction" ]
|
||||
verbs: [ "create" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "configmaps" ]
|
||||
verbs: ["get", "list", "watch" ]
|
||||
- apiGroups: [ "" ]
|
||||
resources: [ "events" ]
|
||||
verbs: [ "list", "watch", "create", "update", "patch" ]
|
||||
---
|
||||
# Source: volcano/templates/agent.yaml
|
||||
kind: ClusterRoleBinding
|
||||
apiVersion: rbac.authorization.k8s.io/v1
|
||||
metadata:
|
||||
labels:
|
||||
app: volcano-agent
|
||||
name: volcano-agent-role
|
||||
subjects:
|
||||
- kind: ServiceAccount
|
||||
name: volcano-agent
|
||||
namespace: volcano-system
|
||||
- kind: User
|
||||
apiGroup: rbac.authorization.k8s.io
|
||||
name: volcano-agent
|
||||
roleRef:
|
||||
kind: ClusterRole
|
||||
name: volcano-agent
|
||||
apiGroup: rbac.authorization.k8s.io
|
||||
---
|
||||
# Source: volcano/templates/agent.yaml
|
||||
apiVersion: apps/v1
|
||||
kind: DaemonSet
|
||||
metadata:
|
||||
name: volcano-agent
|
||||
namespace: volcano-system
|
||||
spec:
|
||||
selector:
|
||||
matchLabels:
|
||||
name: volcano-agent
|
||||
template:
|
||||
metadata:
|
||||
name: volcano-agent
|
||||
labels:
|
||||
name: volcano-agent
|
||||
annotations:
|
||||
prometheus.io/path: /metrics
|
||||
prometheus.io/port: '3300'
|
||||
prometheus.io/scheme: http
|
||||
prometheus.io/scrape: 'true'
|
||||
spec:
|
||||
tolerations:
|
||||
- effect: NoSchedule
|
||||
key: volcano.sh/offline-job-evicting
|
||||
operator: Exists
|
||||
serviceAccountName: volcano-agent
|
||||
hostNetwork: true
|
||||
priorityClassName: system-node-critical
|
||||
restartPolicy: Always
|
||||
dnsPolicy: Default
|
||||
volumes:
|
||||
- name: bwm-dir
|
||||
hostPath:
|
||||
path: /usr/share/bwmcli/
|
||||
type: DirectoryOrCreate
|
||||
- name: cni-plugin-dir
|
||||
hostPath:
|
||||
path: /opt/cni/bin
|
||||
type: Directory
|
||||
- name: host-etc
|
||||
hostPath:
|
||||
path: /etc
|
||||
type: Directory
|
||||
- name: host-sys-fs
|
||||
hostPath:
|
||||
path: /sys/fs
|
||||
type: Directory
|
||||
- name: host-proc-sys
|
||||
hostPath:
|
||||
path: /proc/sys
|
||||
type: Directory
|
||||
- name: log
|
||||
hostPath:
|
||||
path: /var/log/volcano/agent
|
||||
type: ''
|
||||
- name: localtime
|
||||
hostPath:
|
||||
path: /etc/localtime
|
||||
type: ''
|
||||
- name: kubelet-cpu-manager-policy
|
||||
hostPath:
|
||||
path: /var/lib/kubelet/
|
||||
type: ''
|
||||
- name: proc-stat
|
||||
hostPath:
|
||||
path: /proc/stat
|
||||
type: File
|
||||
initContainers:
|
||||
- name: volcano-agent-init
|
||||
image: docker.io/volcanosh/vc-agent:latest
|
||||
command:
|
||||
- /bin/sh
|
||||
- '-c'
|
||||
- /usr/local/bin/install.sh
|
||||
volumeMounts:
|
||||
- name: bwm-dir
|
||||
mountPath: /usr/share/bwmcli
|
||||
- name: cni-plugin-dir
|
||||
mountPath: /opt/cni/bin
|
||||
- name: host-etc
|
||||
mountPath: /host/etc
|
||||
- mountPath: /var/log/volcano/agent
|
||||
name: log
|
||||
- name: host-proc-sys
|
||||
mountPath: /host/proc/sys
|
||||
terminationMessagePath: /dev/termination-log
|
||||
terminationMessagePolicy: File
|
||||
imagePullPolicy: Always
|
||||
containers:
|
||||
- name: volcano-agent
|
||||
image: docker.io/volcanosh/vc-agent:latest
|
||||
command:
|
||||
- /bin/sh
|
||||
- '-c'
|
||||
- /vc-agent --v=2 1>>/var/log/volcano/agent/volcano-agent.log 2>&1
|
||||
env:
|
||||
- name: SYS_FS_PATH
|
||||
value: /host/sys/fs
|
||||
- name: CNI_CONF_FILE_PATH
|
||||
value: /host/etc/cni/net.d/cni.conflist
|
||||
- name: KUBE_NODE_NAME
|
||||
valueFrom:
|
||||
fieldRef:
|
||||
apiVersion: v1
|
||||
fieldPath: spec.nodeName
|
||||
- name: KUBE_POD_NAME
|
||||
valueFrom:
|
||||
fieldRef:
|
||||
apiVersion: v1
|
||||
fieldPath: metadata.name
|
||||
- name: KUBE_POD_NAMESPACE
|
||||
valueFrom:
|
||||
fieldRef:
|
||||
apiVersion: v1
|
||||
fieldPath: metadata.namespace
|
||||
volumeMounts:
|
||||
- name: bwm-dir
|
||||
mountPath: /usr/share/bwmcli
|
||||
- name: cni-plugin-dir
|
||||
mountPath: /opt/cni/bin
|
||||
- name: host-etc
|
||||
mountPath: /host/etc
|
||||
- mountPath: /var/log/volcano/agent
|
||||
name: log
|
||||
- name: host-sys-fs
|
||||
mountPath: /host/sys/fs
|
||||
mountPropagation: HostToContainer
|
||||
- name: host-proc-sys
|
||||
mountPath: /host/proc/sys
|
||||
- name: localtime
|
||||
readOnly: true
|
||||
mountPath: /etc/localtime
|
||||
- name: kubelet-cpu-manager-policy
|
||||
readOnly: true
|
||||
mountPath: /var/lib/kubelet
|
||||
- name: proc-stat
|
||||
readOnly: true
|
||||
mountPath: /host/proc/stat
|
||||
livenessProbe:
|
||||
httpGet:
|
||||
path: /healthz
|
||||
port: 3300
|
||||
scheme: HTTP
|
||||
initialDelaySeconds: 60
|
||||
timeoutSeconds: 5
|
||||
periodSeconds: 10
|
||||
successThreshold: 1
|
||||
failureThreshold: 5
|
||||
imagePullPolicy: Always
|
||||
updateStrategy:
|
||||
type: RollingUpdate
|
||||
rollingUpdate:
|
||||
maxUnavailable: 10%
|
||||
maxSurge: 0
|
||||
revisionHistoryLimit: 10
|
||||
|
|
@ -1,6 +1,6 @@
|
|||
The MIT License (MIT)
|
||||
|
||||
Copyright (c) 2014 Alex Saskevich
|
||||
Copyright (c) 2014-2020 Alex Saskevich
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining a copy
|
||||
of this software and associated documentation files (the "Software"), to deal
|
||||
|
|
|
|||
|
|
@ -0,0 +1,27 @@
|
|||
Copyright (c) 2014 Will Fitzgerald. All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or without
|
||||
modification, are permitted provided that the following conditions are
|
||||
met:
|
||||
|
||||
* Redistributions of source code must retain the above copyright
|
||||
notice, this list of conditions and the following disclaimer.
|
||||
* Redistributions in binary form must reproduce the above
|
||||
copyright notice, this list of conditions and the following disclaimer
|
||||
in the documentation and/or other materials provided with the
|
||||
distribution.
|
||||
* Neither the name of Google Inc. nor the names of its
|
||||
contributors may be used to endorse or promote products derived from
|
||||
this software without specific prior written permission.
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
|
@ -0,0 +1,23 @@
|
|||
MIT License
|
||||
|
||||
Copyright (c) 2017 Nathan Sweet
|
||||
Copyright (c) 2018, 2019 Cloudflare
|
||||
Copyright (c) 2019 Authors of Cilium
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining a copy
|
||||
of this software and associated documentation files (the "Software"), to deal
|
||||
in the Software without restriction, including without limitation the rights
|
||||
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
|
||||
copies of the Software, and to permit persons to whom the Software is
|
||||
furnished to do so, subject to the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be included in all
|
||||
copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
|
||||
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
|
||||
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
|
||||
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
|
||||
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
|
||||
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
|
||||
SOFTWARE.
|
||||
|
|
@ -0,0 +1,202 @@
|
|||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "{}"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright {yyyy} {name of copyright owner}
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
||||
|
|
@ -0,0 +1,201 @@
|
|||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "{}"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright {yyyy} {name of copyright owner}
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
|
@ -0,0 +1,28 @@
|
|||
Copyright (C) 2014-2015 Docker Inc & Go Authors. All rights reserved.
|
||||
Copyright (C) 2017 SUSE LLC. All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or without
|
||||
modification, are permitted provided that the following conditions are
|
||||
met:
|
||||
|
||||
* Redistributions of source code must retain the above copyright
|
||||
notice, this list of conditions and the following disclaimer.
|
||||
* Redistributions in binary form must reproduce the above
|
||||
copyright notice, this list of conditions and the following disclaimer
|
||||
in the documentation and/or other materials provided with the
|
||||
distribution.
|
||||
* Neither the name of Google Inc. nor the names of its
|
||||
contributors may be used to endorse or promote products derived from
|
||||
this software without specific prior written permission.
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
|
@ -0,0 +1,25 @@
|
|||
Copyright (c) 2013, Georg Reinke (<guelfey at gmail dot com>), Google
|
||||
All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or without
|
||||
modification, are permitted provided that the following conditions
|
||||
are met:
|
||||
|
||||
1. Redistributions of source code must retain the above copyright notice,
|
||||
this list of conditions and the following disclaimer.
|
||||
|
||||
2. Redistributions in binary form must reproduce the above copyright
|
||||
notice, this list of conditions and the following disclaimer in the
|
||||
documentation and/or other materials provided with the distribution.
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
|
||||
TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
|
||||
PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
|
||||
LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
|
||||
NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
|
||||
SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
|
@ -0,0 +1,191 @@
|
|||
|
||||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
Copyright 2015 The Linux Foundation.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
|
@ -0,0 +1,201 @@
|
|||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "[]"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright [yyyy] [name of copyright owner]
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
|
@ -0,0 +1,21 @@
|
|||
The MIT License (MIT)
|
||||
|
||||
Copyright (c) 2014 Simon Eskildsen
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining a copy
|
||||
of this software and associated documentation files (the "Software"), to deal
|
||||
in the Software without restriction, including without limitation the rights
|
||||
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
|
||||
copies of the Software, and to permit persons to whom the Software is
|
||||
furnished to do so, subject to the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be included in
|
||||
all copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
|
||||
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
|
||||
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
|
||||
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
|
||||
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
|
||||
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
|
||||
THE SOFTWARE.
|
||||
|
|
@ -0,0 +1,192 @@
|
|||
|
||||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
Copyright 2014 Vishvananda Ishaya.
|
||||
Copyright 2014 Docker, Inc.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
|
@ -0,0 +1,192 @@
|
|||
|
||||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
Copyright 2014 Vishvananda Ishaya.
|
||||
Copyright 2014 Docker, Inc.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
|
@ -0,0 +1,201 @@
|
|||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "{}"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright {yyyy} {name of copyright owner}
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
|
@ -0,0 +1,201 @@
|
|||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "{}"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright {yyyy} {name of copyright owner}
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
|
@ -0,0 +1,62 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package extension
|
||||
|
||||
import (
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
)
|
||||
|
||||
type QosLevel string
|
||||
|
||||
const (
|
||||
// QosLevelLC means the qos level Latency Critical.
|
||||
QosLevelLC QosLevel = "LC"
|
||||
// QosLevelHLS means the qos level Highly Latency Sensitive.
|
||||
QosLevelHLS QosLevel = "HLS"
|
||||
// QosLevelLS means the qos level Latency Sensitive.
|
||||
QosLevelLS QosLevel = "LS"
|
||||
// QosLevelBE means the qos level Best Effort.
|
||||
QosLevelBE QosLevel = "BE"
|
||||
)
|
||||
|
||||
var qosLevelMap = map[QosLevel]int{
|
||||
QosLevelLC: 2,
|
||||
QosLevelHLS: 2,
|
||||
QosLevelLS: 1,
|
||||
QosLevelBE: -1,
|
||||
}
|
||||
|
||||
// GetQosLevel return OS qos level by QosLevel.
|
||||
// If not specified, zero will be returned.
|
||||
func GetQosLevel(pod *corev1.Pod) int {
|
||||
if pod == nil {
|
||||
return 0
|
||||
}
|
||||
|
||||
qosLevel := pod.GetAnnotations()[apis.PodQosLevelKey]
|
||||
return qosLevelMap[QosLevel(qosLevel)]
|
||||
}
|
||||
|
||||
// NormalizeQosLevel normalizes qos level, for memory and network qos, only 0 and -1 are supported now.
|
||||
func NormalizeQosLevel(qosLevel int64) int64 {
|
||||
if qosLevel < 0 {
|
||||
return -1
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package extension
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
)
|
||||
|
||||
func TestGetQosLevel(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
pod *corev1.Pod
|
||||
want int
|
||||
}{
|
||||
{
|
||||
name: "no qos level specified",
|
||||
pod: makePod(""),
|
||||
want: 0,
|
||||
},
|
||||
{
|
||||
name: "invalid qos level",
|
||||
pod: makePod("invalid"),
|
||||
want: 0,
|
||||
},
|
||||
{
|
||||
name: "qos LC",
|
||||
pod: makePod("LC"),
|
||||
want: 2,
|
||||
},
|
||||
{
|
||||
name: "qos HLS",
|
||||
pod: makePod("HLS"),
|
||||
want: 2,
|
||||
},
|
||||
{
|
||||
name: "qos LS",
|
||||
pod: makePod("LS"),
|
||||
want: 1,
|
||||
},
|
||||
{
|
||||
name: "qos BE",
|
||||
pod: makePod("BE"),
|
||||
want: -1,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
if got := GetQosLevel(tt.pod); got != tt.want {
|
||||
t.Errorf("GetQosLevel() = %v, want %v", got, tt.want)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func makePod(qosLevel string) *corev1.Pod {
|
||||
return &corev1.Pod{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Annotations: map[string]string{
|
||||
"volcano.sh/qos-level": qosLevel,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package apis
|
||||
|
||||
import corev1 "k8s.io/api/core/v1"
|
||||
|
||||
const (
|
||||
// OverSubscriptionTypesKey define the overSubscription resource types
|
||||
OverSubscriptionTypesKey = "volcano.sh/oversubscription-types"
|
||||
|
||||
// NodeOverSubscriptionCPUKey define the oversubscription cpu resource on the node
|
||||
NodeOverSubscriptionCPUKey = "volcano.sh/oversubscription-cpu"
|
||||
// NodeOverSubscriptionMemoryKey define the oversubscription memory resource on the node
|
||||
NodeOverSubscriptionMemoryKey = "volcano.sh/oversubscription-memory"
|
||||
|
||||
// PodQosLevelKey define pod qos level, see pkg/agent/apis/extension/qos.go for specific values.
|
||||
PodQosLevelKey = "volcano.sh/qos-level"
|
||||
// PodEvictingKey define if the offline job is evicting
|
||||
PodEvictingKey = "volcano.sh/offline-job-evicting"
|
||||
// ColocationEnableNodeLabelKey is the label name for colocation,
|
||||
// indicates whether the node enable colocation, set "true" or "false", default is "false"
|
||||
ColocationEnableNodeLabelKey = "volcano.sh/colocation"
|
||||
// OverSubscriptionNodeLabelKey define whether a node is oversubscription node.
|
||||
OverSubscriptionNodeLabelKey = "volcano.sh/oversubscription"
|
||||
|
||||
// NetworkBandwidthRateAnnotationKey is the annotation key of network bandwidth rate, unit Mbps.
|
||||
NetworkBandwidthRateAnnotationKey = "volcano.sh/network-bandwidth-rate"
|
||||
|
||||
// Deprecated:This is used to be compatible with old api.
|
||||
// PodEvictedOverSubscriptionCPUHighWaterMarkKey define the high watermark of cpu usage when evicting offline pods
|
||||
PodEvictedOverSubscriptionCPUHighWaterMarkKey = "volcano.sh/oversubscription-evicting-cpu-high-watermark"
|
||||
// Deprecated:This is used to be compatible with old api.
|
||||
// PodEvictedOverSubscriptionMemoryHighWaterMarkKey define the high watermark of memory usage when evicting offline pods
|
||||
PodEvictedOverSubscriptionMemoryHighWaterMarkKey = "volcano.sh/oversubscription-evicting-memory-high-watermark"
|
||||
// Deprecated:This is used to be compatible with old api.
|
||||
// PodEvictedOverSubscriptionCPULowWaterMarkKey define the low watermark of cpu usage when the node could overSubscription resources
|
||||
PodEvictedOverSubscriptionCPULowWaterMarkKey = "volcano.sh/oversubscription-evicting-cpu-low-watermark"
|
||||
// Deprecated:This is used to be compatible with old api.
|
||||
// PodEvictedOverSubscriptionMemoryLowWaterMarkKey define the low watermark of memory usage when the node could overSubscription resources
|
||||
PodEvictedOverSubscriptionMemoryLowWaterMarkKey = "volcano.sh/oversubscription-evicting-memory-low-watermark"
|
||||
|
||||
// PodEvictedCPUHighWaterMarkKey define the high watermark of cpu usage when evicting offline pods
|
||||
PodEvictedCPUHighWaterMarkKey = "volcano.sh/evicting-cpu-high-watermark"
|
||||
// PodEvictedMemoryHighWaterMarkKey define the high watermark of memory usage when evicting offline pods
|
||||
PodEvictedMemoryHighWaterMarkKey = "volcano.sh/evicting-memory-high-watermark"
|
||||
// PodEvictedCPULowWaterMarkKey define the low watermark of cpu usage when the node could overSubscription resources
|
||||
PodEvictedCPULowWaterMarkKey = "volcano.sh/evicting-cpu-low-watermark"
|
||||
// PodEvictedMemoryLowWaterMarkKey define the low watermark of memory usage when the node could overSubscription resources
|
||||
PodEvictedMemoryLowWaterMarkKey = "volcano.sh/evicting-memory-low-watermark"
|
||||
|
||||
// ResourceDefaultPrefix is the extended resource prefix.
|
||||
ResourceDefaultPrefix = "kubernetes.io/"
|
||||
|
||||
ExtendResourceCPU = ResourceDefaultPrefix + "batch-cpu"
|
||||
ExtendResourceMemory = ResourceDefaultPrefix + "batch-memory"
|
||||
|
||||
// ColocationPolicyKey is the label key of node custom colocation policy.
|
||||
ColocationPolicyKey = "colocation-policy"
|
||||
)
|
||||
|
||||
var OverSubscriptionResourceTypes = []corev1.ResourceName{corev1.ResourceCPU, corev1.ResourceMemory}
|
||||
|
||||
var OverSubscriptionResourceTypesIncludeExtendResources = []corev1.ResourceName{corev1.ResourceCPU, corev1.ResourceMemory, ExtendResourceCPU, ExtendResourceMemory}
|
||||
|
||||
// Resource mapping resource type and usage.
|
||||
type Resource map[corev1.ResourceName]int64
|
||||
|
||||
// Watermark defines resource eviction watermark.
|
||||
type Watermark map[corev1.ResourceName]int
|
||||
|
|
@ -0,0 +1,113 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package api
|
||||
|
||||
import (
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
)
|
||||
|
||||
// VolcanoAgentConfig include global and node colocation config.
|
||||
type VolcanoAgentConfig struct {
|
||||
// GlobalConfig is a global config for all nodes.
|
||||
GlobalConfig *ColocationConfig `json:"globalConfig,omitempty"`
|
||||
|
||||
// NodesConfig will overwrite GlobalConfig for selector matched nodes, which is usually nodePool level.
|
||||
NodesConfig []NodesConfig `json:"nodesConfig,omitempty"`
|
||||
}
|
||||
|
||||
// NodeLabelConfig does not support getting from configmap
|
||||
type NodeLabelConfig struct {
|
||||
// NodeColocationEnable enables node colocation or not.
|
||||
NodeColocationEnable *bool
|
||||
// NodeColocationEnable enables node oversubscription or not.
|
||||
NodeOverSubscriptionEnable *bool
|
||||
}
|
||||
|
||||
type NodesConfig struct {
|
||||
// nodes that match label selector will apply current configuration
|
||||
Selector *metav1.LabelSelector `json:"selector,omitempty"`
|
||||
ColocationConfig `json:",inline"`
|
||||
}
|
||||
|
||||
type ColocationConfig struct {
|
||||
// got from node labels
|
||||
NodeLabelConfig *NodeLabelConfig `json:"-"`
|
||||
|
||||
// cpu qos related config.
|
||||
CPUQosConfig *CPUQos `json:"cpuQosConfig,omitempty" configKey:"CPUQoS"`
|
||||
|
||||
// cpu burst related config.
|
||||
CPUBurstConfig *CPUBurst `json:"cpuBurstConfig,omitempty" configKey:"CPUBurst"`
|
||||
|
||||
// memory qos related config.
|
||||
MemoryQosConfig *MemoryQos `json:"memoryQosConfig,omitempty" configKey:"MemoryQoS"`
|
||||
|
||||
// network qos related config.
|
||||
NetworkQosConfig *NetworkQos `json:"networkQosConfig,omitempty" configKey:"NetworkQoS"`
|
||||
|
||||
// overSubscription related config.
|
||||
OverSubscriptionConfig *OverSubscription `json:"overSubscriptionConfig,omitempty" configKey:"OverSubscription"`
|
||||
|
||||
// Evicting related config.
|
||||
EvictingConfig *Evicting `json:"evictingConfig,omitempty" configKey:"Evicting"`
|
||||
}
|
||||
|
||||
type CPUQos struct {
|
||||
// Enable CPUQos or not.
|
||||
Enable *bool `json:"enable,omitempty"`
|
||||
}
|
||||
|
||||
type CPUBurst struct {
|
||||
// Enable CPUBurst or not.
|
||||
Enable *bool `json:"enable,omitempty"`
|
||||
}
|
||||
|
||||
type MemoryQos struct {
|
||||
// Enable MemoryQos or not.
|
||||
Enable *bool `json:"enable,omitempty"`
|
||||
}
|
||||
|
||||
type NetworkQos struct {
|
||||
// Enable NetworkQos or not.
|
||||
Enable *bool `json:"enable,omitempty"`
|
||||
// OnlineBandwidthWatermarkPercent presents the online bandwidth threshold percent.
|
||||
OnlineBandwidthWatermarkPercent *int `json:"onlineBandwidthWatermarkPercent,omitempty"`
|
||||
// OfflineLowBandwidthPercent presents the offline low bandwidth threshold percent.
|
||||
OfflineLowBandwidthPercent *int `json:"offlineLowBandwidthPercent,omitempty"`
|
||||
// OfflineHighBandwidthPercent presents the offline high bandwidth threshold percent.
|
||||
OfflineHighBandwidthPercent *int `json:"offlineHighBandwidthPercent,omitempty"`
|
||||
// QoSCheckInterval presents the network Qos checkout interval
|
||||
QoSCheckInterval *int `json:"qosCheckInterval,omitempty"`
|
||||
}
|
||||
|
||||
type OverSubscription struct {
|
||||
// Enable OverSubscription or not.
|
||||
Enable *bool `json:"enable,omitempty"`
|
||||
// OverSubscriptionTypes defines over subscription types, such as cpu,memory.
|
||||
OverSubscriptionTypes *string `json:"overSubscriptionTypes,omitempty"`
|
||||
}
|
||||
|
||||
type Evicting struct {
|
||||
// EvictingCPUHighWatermark defines the high watermark percent of cpu usage when evicting offline pods.
|
||||
EvictingCPUHighWatermark *int `json:"evictingCPUHighWatermark,omitempty"`
|
||||
// EvictingMemoryHighWatermark defines the high watermark percent of memory usage when evicting offline pods.
|
||||
EvictingMemoryHighWatermark *int `json:"evictingMemoryHighWatermark,omitempty"`
|
||||
// EvictingCPULowWatermark defines the low watermark percent of cpu usage when the node recover schedule pods.
|
||||
EvictingCPULowWatermark *int `json:"evictingCPULowWatermark,omitempty"`
|
||||
// EvictingMemoryLowWatermark defines the low watermark percent of memory usage when the node could recover schedule pods.
|
||||
EvictingMemoryLowWatermark *int `json:"evictingMemoryLowWatermark,omitempty"`
|
||||
}
|
||||
|
|
@ -0,0 +1,146 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package api
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"strings"
|
||||
)
|
||||
|
||||
var (
|
||||
IllegalQoSCheckIntervalMsg = "qosCheckInterval must be a positive number"
|
||||
IllegalOnlineBandwidthWatermarkPercentMsg = "onlineBandwidthWatermarkPercent must be a positive number between 1 and 1000"
|
||||
IllegalOfflineHighBandwidthPercentMsg = "offlineHighBandwidthPercent must be a positive number between 1 and 1000"
|
||||
IllegalOfflineLowBandwidthPercentMsg = "offlineLowBandwidthPercent must be a positive number between 1 and 1000"
|
||||
OfflineHighBandwidthPercentLessOfflineLowBandwidthPercentMsg = "offlineHighBandwidthPercent cannot be less than offlineLowBandwidthPercent"
|
||||
IllegalEvictingCPUHighWatermark = "evictingCPUHighWatermark must be a positive number"
|
||||
IllegalEvictingMemoryHighWatermark = "evictingMemoryHighWatermark must be a positive number"
|
||||
IllegalEvictingCPULowWatermark = "evictingCPULowWatermark must be a positive number"
|
||||
IllegalEvictingMemoryLowWatermark = "evictingMemoryLowWatermark must be a positive number"
|
||||
EvictingCPULowWatermarkHigherThanHighWatermark = "cpu evicting low watermark is higher than high watermark"
|
||||
EvictingMemoryLowWatermarkHigherThanHighWatermark = "memory evicting low watermark is higher than high watermark"
|
||||
IllegalOverSubscriptionTypes = "overSubscriptionType(%s) is not supported, only supports cpu/memory"
|
||||
)
|
||||
|
||||
type Validate interface {
|
||||
Validate() []error
|
||||
}
|
||||
|
||||
func (n *NodeLabelConfig) Validate() []error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *CPUQos) Validate() []error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *CPUBurst) Validate() []error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *MemoryQos) Validate() []error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (n *NetworkQos) Validate() []error {
|
||||
if n == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
var errs []error
|
||||
if n.QoSCheckInterval != nil && *n.QoSCheckInterval <= 0 {
|
||||
errs = append(errs, errors.New(IllegalQoSCheckIntervalMsg))
|
||||
}
|
||||
|
||||
// Maximum Bandwidth may be several times the Baseline Bandwidth.
|
||||
// Use 1000 ( 10 times the Baseline Bandwidth ) as the upper limit.
|
||||
if n.OnlineBandwidthWatermarkPercent != nil && (*n.OnlineBandwidthWatermarkPercent <= 0 || *n.OnlineBandwidthWatermarkPercent > 1000) {
|
||||
errs = append(errs, errors.New(IllegalOnlineBandwidthWatermarkPercentMsg))
|
||||
}
|
||||
if n.OfflineHighBandwidthPercent != nil && (*n.OfflineHighBandwidthPercent <= 0 || *n.OfflineHighBandwidthPercent > 1000) {
|
||||
errs = append(errs, errors.New(IllegalOfflineHighBandwidthPercentMsg))
|
||||
}
|
||||
if n.OfflineLowBandwidthPercent != nil && (*n.OfflineLowBandwidthPercent <= 0 || *n.OfflineLowBandwidthPercent > 1000) {
|
||||
errs = append(errs, errors.New(IllegalOfflineLowBandwidthPercentMsg))
|
||||
}
|
||||
if n.OfflineLowBandwidthPercent != nil && n.OfflineHighBandwidthPercent != nil && (*n.OfflineLowBandwidthPercent > *n.OfflineHighBandwidthPercent) {
|
||||
errs = append(errs, errors.New(OfflineHighBandwidthPercentLessOfflineLowBandwidthPercentMsg))
|
||||
}
|
||||
return errs
|
||||
}
|
||||
|
||||
func (o *OverSubscription) Validate() []error {
|
||||
if o == nil {
|
||||
return nil
|
||||
}
|
||||
if o.OverSubscriptionTypes == nil || len(*o.OverSubscriptionTypes) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
var errs []error
|
||||
types := strings.Split(*o.OverSubscriptionTypes, ",")
|
||||
for _, t := range types {
|
||||
if t != "cpu" && t != "memory" {
|
||||
errs = append(errs, fmt.Errorf(IllegalOverSubscriptionTypes, t))
|
||||
}
|
||||
}
|
||||
return errs
|
||||
}
|
||||
|
||||
func (e *Evicting) Validate() []error {
|
||||
if e == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
var errs []error
|
||||
if e.EvictingCPUHighWatermark != nil && *e.EvictingCPUHighWatermark <= 0 {
|
||||
errs = append(errs, errors.New(IllegalEvictingCPUHighWatermark))
|
||||
}
|
||||
if e.EvictingMemoryHighWatermark != nil && *e.EvictingMemoryHighWatermark <= 0 {
|
||||
errs = append(errs, errors.New(IllegalEvictingMemoryHighWatermark))
|
||||
}
|
||||
if e.EvictingCPULowWatermark != nil && *e.EvictingCPULowWatermark <= 0 {
|
||||
errs = append(errs, errors.New(IllegalEvictingCPULowWatermark))
|
||||
}
|
||||
if e.EvictingMemoryLowWatermark != nil && *e.EvictingMemoryLowWatermark <= 0 {
|
||||
errs = append(errs, errors.New(IllegalEvictingMemoryLowWatermark))
|
||||
}
|
||||
if e.EvictingCPULowWatermark != nil && e.EvictingCPUHighWatermark != nil && (*e.EvictingCPULowWatermark > *e.EvictingCPUHighWatermark) {
|
||||
errs = append(errs, errors.New(EvictingCPULowWatermarkHigherThanHighWatermark))
|
||||
}
|
||||
if e.EvictingMemoryLowWatermark != nil && e.EvictingMemoryHighWatermark != nil && (*e.EvictingMemoryLowWatermark > *e.EvictingMemoryHighWatermark) {
|
||||
errs = append(errs, errors.New(EvictingMemoryLowWatermarkHigherThanHighWatermark))
|
||||
}
|
||||
return errs
|
||||
}
|
||||
|
||||
func (c *ColocationConfig) Validate() []error {
|
||||
if c == nil {
|
||||
return nil
|
||||
}
|
||||
var errs []error
|
||||
|
||||
errs = append(errs, c.NodeLabelConfig.Validate()...)
|
||||
errs = append(errs, c.CPUQosConfig.Validate()...)
|
||||
errs = append(errs, c.CPUBurstConfig.Validate()...)
|
||||
errs = append(errs, c.MemoryQosConfig.Validate()...)
|
||||
errs = append(errs, c.NetworkQosConfig.Validate()...)
|
||||
errs = append(errs, c.OverSubscriptionConfig.Validate()...)
|
||||
errs = append(errs, c.EvictingConfig.Validate()...)
|
||||
return errs
|
||||
}
|
||||
|
|
@ -0,0 +1,149 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package api
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
utilpointer "k8s.io/utils/pointer"
|
||||
)
|
||||
|
||||
func TestColocationConfigValidate(t *testing.T) {
|
||||
testCases := []struct {
|
||||
name string
|
||||
colocationCfg *ColocationConfig
|
||||
expectedErr []error
|
||||
}{
|
||||
{
|
||||
name: "legal configuration",
|
||||
colocationCfg: &ColocationConfig{
|
||||
NodeLabelConfig: &NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(false),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(false),
|
||||
},
|
||||
CPUQosConfig: &CPUQos{Enable: utilpointer.Bool(true)},
|
||||
CPUBurstConfig: &CPUBurst{Enable: utilpointer.Bool(true)},
|
||||
MemoryQosConfig: &MemoryQos{Enable: utilpointer.Bool(true)},
|
||||
NetworkQosConfig: &NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(10),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(10),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(20),
|
||||
QoSCheckInterval: utilpointer.Int(100),
|
||||
},
|
||||
OverSubscriptionConfig: &OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String("cpu"),
|
||||
},
|
||||
EvictingConfig: &Evicting{
|
||||
EvictingCPUHighWatermark: utilpointer.Int(20),
|
||||
EvictingMemoryHighWatermark: utilpointer.Int(20),
|
||||
EvictingCPULowWatermark: utilpointer.Int(10),
|
||||
EvictingMemoryLowWatermark: utilpointer.Int(10),
|
||||
},
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
name: "illegal NetworkQosConfig && negative parameters",
|
||||
colocationCfg: &ColocationConfig{
|
||||
NetworkQosConfig: &NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(-10),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(-20),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(-20),
|
||||
QoSCheckInterval: utilpointer.Int(-100),
|
||||
},
|
||||
},
|
||||
expectedErr: []error{errors.New(IllegalQoSCheckIntervalMsg), errors.New(IllegalOnlineBandwidthWatermarkPercentMsg),
|
||||
errors.New(IllegalOfflineHighBandwidthPercentMsg), errors.New(IllegalOfflineLowBandwidthPercentMsg)},
|
||||
},
|
||||
|
||||
{
|
||||
name: "illegal NetworkQosConfig && OfflineLowBandwidthPercent higher than OfflineHighBandwidthPercent",
|
||||
colocationCfg: &ColocationConfig{
|
||||
NetworkQosConfig: &NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(10),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(20),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(10),
|
||||
QoSCheckInterval: utilpointer.Int(100),
|
||||
},
|
||||
},
|
||||
expectedErr: []error{errors.New(OfflineHighBandwidthPercentLessOfflineLowBandwidthPercentMsg)},
|
||||
},
|
||||
|
||||
{
|
||||
name: "illegal NetworkQosConfig && OfflineLowBandwidthPercent higher than OfflineHighBandwidthPercent",
|
||||
colocationCfg: &ColocationConfig{
|
||||
NetworkQosConfig: &NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(10),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(20),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(10),
|
||||
QoSCheckInterval: utilpointer.Int(100),
|
||||
},
|
||||
},
|
||||
expectedErr: []error{errors.New(OfflineHighBandwidthPercentLessOfflineLowBandwidthPercentMsg)},
|
||||
},
|
||||
|
||||
{
|
||||
name: "illegal EvictingConfig && negative parameters",
|
||||
colocationCfg: &ColocationConfig{
|
||||
EvictingConfig: &Evicting{
|
||||
EvictingCPUHighWatermark: utilpointer.Int(-10),
|
||||
EvictingMemoryHighWatermark: utilpointer.Int(-10),
|
||||
EvictingCPULowWatermark: utilpointer.Int(-20),
|
||||
EvictingMemoryLowWatermark: utilpointer.Int(-20),
|
||||
},
|
||||
},
|
||||
expectedErr: []error{errors.New(IllegalEvictingCPUHighWatermark), errors.New(IllegalEvictingMemoryHighWatermark),
|
||||
errors.New(IllegalEvictingCPULowWatermark), errors.New(IllegalEvictingMemoryLowWatermark)},
|
||||
},
|
||||
|
||||
{
|
||||
name: "illegal OverSubscriptionConfig",
|
||||
colocationCfg: &ColocationConfig{
|
||||
OverSubscriptionConfig: &OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String("fake"),
|
||||
},
|
||||
},
|
||||
expectedErr: []error{fmt.Errorf(IllegalOverSubscriptionTypes, "fake")},
|
||||
},
|
||||
{
|
||||
name: "cpu evicting low watermark higher high watermark",
|
||||
colocationCfg: &ColocationConfig{
|
||||
EvictingConfig: &Evicting{
|
||||
EvictingCPUHighWatermark: utilpointer.Int(20),
|
||||
EvictingMemoryHighWatermark: utilpointer.Int(30),
|
||||
EvictingCPULowWatermark: utilpointer.Int(50),
|
||||
EvictingMemoryLowWatermark: utilpointer.Int(80),
|
||||
},
|
||||
},
|
||||
expectedErr: []error{errors.New(EvictingCPULowWatermarkHigherThanHighWatermark), errors.New(EvictingMemoryLowWatermarkHigherThanHighWatermark)},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
actualErrs := tc.colocationCfg.Validate()
|
||||
assert.Equal(t, tc.expectedErr, actualErrs)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,241 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package config
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/errors"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
utilerrors "k8s.io/apimachinery/pkg/util/errors"
|
||||
"k8s.io/apimachinery/pkg/util/wait"
|
||||
clientset "k8s.io/client-go/kubernetes"
|
||||
listersv1 "k8s.io/client-go/listers/core/v1"
|
||||
"k8s.io/client-go/tools/record"
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
"k8s.io/klog/v2"
|
||||
utilpointer "k8s.io/utils/pointer"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/config/source"
|
||||
"volcano.sh/volcano/pkg/agent/config/utils"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
)
|
||||
|
||||
type Listener interface {
|
||||
SyncConfig(cfg *api.ColocationConfig) error
|
||||
}
|
||||
|
||||
type ConfigManager struct {
|
||||
kubeClient clientset.Interface
|
||||
configmapNamespace string
|
||||
configmapName string
|
||||
agentPodNamespace string
|
||||
agentPodName string
|
||||
source source.ConfigEventSource
|
||||
listeners []Listener
|
||||
podLister listersv1.PodLister
|
||||
recorder record.EventRecorder
|
||||
}
|
||||
|
||||
func NewManager(config *config.Configuration, listeners []Listener) *ConfigManager {
|
||||
return &ConfigManager{
|
||||
kubeClient: config.GenericConfiguration.KubeClient,
|
||||
configmapNamespace: config.GenericConfiguration.KubePodNamespace,
|
||||
configmapName: utils.ConfigMapName,
|
||||
agentPodNamespace: config.GenericConfiguration.KubePodNamespace,
|
||||
agentPodName: config.GenericConfiguration.KubePodName,
|
||||
source: source.NewConfigMapSource(config.GenericConfiguration.KubeClient, config.GenericConfiguration.KubeNodeName, config.GenericConfiguration.KubePodNamespace),
|
||||
listeners: listeners,
|
||||
podLister: config.GenericConfiguration.PodLister,
|
||||
recorder: config.GenericConfiguration.Recorder,
|
||||
}
|
||||
}
|
||||
|
||||
func (m *ConfigManager) PrepareConfigmap() error {
|
||||
getCtx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
cm, getErr := m.kubeClient.CoreV1().ConfigMaps(m.configmapNamespace).Get(getCtx, m.configmapName, metav1.GetOptions{})
|
||||
if getErr == nil {
|
||||
klog.InfoS("Configmap already exists", "name", m.configmapName, "namespace", m.configmapNamespace)
|
||||
return m.updateConfigMap(cm)
|
||||
}
|
||||
|
||||
if !errors.IsNotFound(getErr) {
|
||||
klog.ErrorS(getErr, "Failed to get configMap", "name", m.configmapName, "namespace", m.configmapNamespace)
|
||||
return getErr
|
||||
}
|
||||
|
||||
klog.InfoS("configMap not found, will create a new one")
|
||||
var lastCreateErr error
|
||||
waitErr := wait.PollImmediate(200*time.Millisecond, time.Minute, func() (done bool, err error) {
|
||||
_, createErr := m.kubeClient.CoreV1().ConfigMaps(m.configmapNamespace).Create(context.TODO(), &corev1.ConfigMap{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: m.configmapName,
|
||||
Namespace: m.configmapNamespace},
|
||||
Data: map[string]string{
|
||||
utils.ColocationConfigKey: utils.DefaultCfg,
|
||||
}}, metav1.CreateOptions{})
|
||||
if errors.IsAlreadyExists(createErr) {
|
||||
return true, nil
|
||||
}
|
||||
lastCreateErr = createErr
|
||||
return createErr == nil, nil
|
||||
})
|
||||
if waitErr != nil {
|
||||
klog.ErrorS(waitErr, "Failed to wait for creating configMap")
|
||||
return fmt.Errorf("failed to create configmap(%s:%s), waitting error: %v, latest creation error: %v",
|
||||
m.agentPodNamespace, utils.ConfigMapName, waitErr, lastCreateErr)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *ConfigManager) updateConfigMap(cm *corev1.ConfigMap) error {
|
||||
oldData := cm.Data[utils.ColocationConfigKey]
|
||||
c := &api.VolcanoAgentConfig{}
|
||||
if err := json.Unmarshal([]byte(oldData), c); err != nil {
|
||||
return fmt.Errorf("failed to unmarshal old configMap data: %v", err)
|
||||
}
|
||||
|
||||
changed := false
|
||||
if c.GlobalConfig == nil {
|
||||
return fmt.Errorf("empty glaobal config")
|
||||
}
|
||||
if c.GlobalConfig.OverSubscriptionConfig == nil {
|
||||
changed = true
|
||||
c.GlobalConfig.OverSubscriptionConfig = &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String(utils.DefaultOverSubscriptionTypes),
|
||||
}
|
||||
}
|
||||
if c.GlobalConfig.EvictingConfig == nil {
|
||||
changed = true
|
||||
c.GlobalConfig.EvictingConfig = &api.Evicting{
|
||||
EvictingCPUHighWatermark: utilpointer.Int(utils.DefaultEvictingCPUHighWatermark),
|
||||
EvictingMemoryHighWatermark: utilpointer.Int(utils.DefaultEvictingMemoryHighWatermark),
|
||||
EvictingCPULowWatermark: utilpointer.Int(utils.DefaultEvictingCPULowWatermark),
|
||||
EvictingMemoryLowWatermark: utilpointer.Int(utils.DefaultEvictingMemoryLowWatermark),
|
||||
}
|
||||
}
|
||||
if !changed {
|
||||
return nil
|
||||
}
|
||||
|
||||
newData, err := json.MarshalIndent(c, "", " ")
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to marshal configMap: %v", err)
|
||||
}
|
||||
cm.Data[utils.ColocationConfigKey] = string(newData)
|
||||
_, err = m.kubeClient.CoreV1().ConfigMaps(m.configmapNamespace).Update(context.TODO(), cm, metav1.UpdateOptions{})
|
||||
if err == nil {
|
||||
klog.InfoS("Successfully updated volcano agent configMap")
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (m *ConfigManager) Start(ctx context.Context) error {
|
||||
klog.InfoS("Start configuration manager")
|
||||
if err := m.PrepareConfigmap(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
changesQueue, err := m.source.Source(ctx.Done())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err = m.notifyListeners(); err != nil {
|
||||
klog.ErrorS(err, "Failed to notify all listeners, retry by process chanagesQueue")
|
||||
}
|
||||
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
default:
|
||||
if !m.process(changesQueue) {
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
klog.InfoS("Successfully started configuration manager")
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *ConfigManager) process(changedQueue workqueue.RateLimitingInterface) bool {
|
||||
key, quit := changedQueue.Get()
|
||||
if quit {
|
||||
return false
|
||||
}
|
||||
defer changedQueue.Done(key)
|
||||
|
||||
pod, getErr := m.getAgentPod()
|
||||
if getErr != nil {
|
||||
klog.ErrorS(getErr, "Failed to get agent pod", "namespace", m.agentPodNamespace, "name", m.agentPodName)
|
||||
}
|
||||
|
||||
err := m.notifyListeners()
|
||||
if err != nil {
|
||||
if pod != nil {
|
||||
m.recorder.Event(pod, corev1.EventTypeWarning, "ConfigApplyFailed", fmt.Sprintf("Failed to apply config(%v)", err))
|
||||
}
|
||||
klog.ErrorS(err, "Failed to notify listeners")
|
||||
changedQueue.AddRateLimited(key)
|
||||
return true
|
||||
}
|
||||
|
||||
if pod != nil {
|
||||
m.recorder.Event(pod, corev1.EventTypeNormal, "ConfigApplySuccess", "Successfully applied config")
|
||||
}
|
||||
klog.InfoS("Successfully notified listeners")
|
||||
changedQueue.Forget(key)
|
||||
return true
|
||||
}
|
||||
|
||||
func (m *ConfigManager) notifyListeners() error {
|
||||
config, err := m.source.GetLatestConfig()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
var errs []error
|
||||
for _, listener := range m.listeners {
|
||||
if syncErr := listener.SyncConfig(config); syncErr != nil {
|
||||
errs = append(errs, syncErr)
|
||||
}
|
||||
}
|
||||
return utilerrors.NewAggregate(errs)
|
||||
}
|
||||
|
||||
func (m *ConfigManager) getAgentPod() (*corev1.Pod, error) {
|
||||
pod, err := m.podLister.Pods(m.agentPodNamespace).Get(m.agentPodName)
|
||||
if err == nil {
|
||||
return pod, nil
|
||||
}
|
||||
|
||||
if !errors.IsNotFound(err) {
|
||||
return nil, err
|
||||
}
|
||||
return m.kubeClient.CoreV1().Pods(m.agentPodNamespace).Get(context.TODO(), m.agentPodName, metav1.GetOptions{})
|
||||
}
|
||||
|
|
@ -0,0 +1,307 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package config
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/runtime"
|
||||
"k8s.io/client-go/informers"
|
||||
"k8s.io/client-go/kubernetes/fake"
|
||||
"k8s.io/client-go/tools/cache"
|
||||
"k8s.io/client-go/tools/record"
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
utilpointer "k8s.io/utils/pointer"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/config/source"
|
||||
"volcano.sh/volcano/pkg/agent/config/utils"
|
||||
)
|
||||
|
||||
var oldCfg = `
|
||||
{
|
||||
"globalConfig":{
|
||||
"cpuBurstConfig":{
|
||||
"enable":true
|
||||
},
|
||||
"networkQosConfig":{
|
||||
"enable":true,
|
||||
"onlineBandwidthWatermarkPercent":80,
|
||||
"offlineLowBandwidthPercent":10,
|
||||
"offlineHighBandwidthPercent":40,
|
||||
"qosCheckInterval": 10000000
|
||||
}
|
||||
}
|
||||
}
|
||||
`
|
||||
|
||||
var oldCfgWithSelector = `
|
||||
{
|
||||
"globalConfig":{
|
||||
"cpuBurstConfig":{
|
||||
"enable":true
|
||||
},
|
||||
"networkQosConfig":{
|
||||
"enable":true,
|
||||
"onlineBandwidthWatermarkPercent":80,
|
||||
"offlineLowBandwidthPercent":10,
|
||||
"offlineHighBandwidthPercent":40,
|
||||
"qosCheckInterval": 10000000
|
||||
},
|
||||
"overSubscriptionConfig":{
|
||||
"enable":true,
|
||||
"overSubscriptionTypes":"cpu,memory"
|
||||
},
|
||||
"evictingConfig":{
|
||||
"evictingCPUHighWatermark":80,
|
||||
"evictingMemoryHighWatermark":60,
|
||||
"evictingCPULowWatermark":30,
|
||||
"evictingMemoryLowWatermark":30
|
||||
}
|
||||
},
|
||||
"nodesConfig": [
|
||||
{
|
||||
"selector": {
|
||||
"matchLabels": {
|
||||
"label-key": "label-value"
|
||||
}
|
||||
},
|
||||
"evictingConfig": {
|
||||
"evictingCPUHighWatermark": 60
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
`
|
||||
|
||||
func makeConfigMap(data string) *v1.ConfigMap {
|
||||
return &v1.ConfigMap{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Namespace: "kube-system",
|
||||
Name: "volcano-agent-configuration"},
|
||||
Data: map[string]string{
|
||||
utils.ColocationConfigKey: data,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
var agentPod = &v1.Pod{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Namespace: "kube-system",
|
||||
Name: "volcano-agent-pod"},
|
||||
}
|
||||
|
||||
func defaultCfg() *api.VolcanoAgentConfig {
|
||||
return &api.VolcanoAgentConfig{GlobalConfig: &api.ColocationConfig{
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(true)},
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(utils.DefaultOnlineBandwidthWatermarkPercent),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(utils.DefaultOfflineLowBandwidthPercent),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(utils.DefaultOfflineHighBandwidthPercent),
|
||||
QoSCheckInterval: utilpointer.Int(utils.DefaultNetworkQoSInterval),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String(utils.DefaultOverSubscriptionTypes),
|
||||
},
|
||||
EvictingConfig: &api.Evicting{
|
||||
EvictingCPUHighWatermark: utilpointer.Int(utils.DefaultEvictingCPUHighWatermark),
|
||||
EvictingMemoryHighWatermark: utilpointer.Int(utils.DefaultEvictingMemoryHighWatermark),
|
||||
EvictingCPULowWatermark: utilpointer.Int(utils.DefaultEvictingCPULowWatermark),
|
||||
EvictingMemoryLowWatermark: utilpointer.Int(utils.DefaultEvictingMemoryLowWatermark),
|
||||
},
|
||||
}}
|
||||
}
|
||||
|
||||
func TestPrepareConfigmap(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
initialObjects []runtime.Object
|
||||
expectedErrIsNil bool
|
||||
expectedConfig *api.VolcanoAgentConfig
|
||||
}{
|
||||
{
|
||||
name: "configmap-existed and update",
|
||||
initialObjects: []runtime.Object{makeConfigMap(oldCfg)},
|
||||
expectedErrIsNil: true,
|
||||
expectedConfig: defaultCfg(),
|
||||
},
|
||||
{
|
||||
name: "configmap-existed and no update",
|
||||
initialObjects: []runtime.Object{makeConfigMap(oldCfgWithSelector)},
|
||||
expectedErrIsNil: true,
|
||||
expectedConfig: &api.VolcanoAgentConfig{GlobalConfig: &api.ColocationConfig{
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(true)},
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(utils.DefaultOnlineBandwidthWatermarkPercent),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(utils.DefaultOfflineLowBandwidthPercent),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(utils.DefaultOfflineHighBandwidthPercent),
|
||||
QoSCheckInterval: utilpointer.Int(utils.DefaultNetworkQoSInterval),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String(utils.DefaultOverSubscriptionTypes),
|
||||
},
|
||||
EvictingConfig: &api.Evicting{
|
||||
EvictingCPUHighWatermark: utilpointer.Int(utils.DefaultEvictingCPUHighWatermark),
|
||||
EvictingMemoryHighWatermark: utilpointer.Int(utils.DefaultEvictingMemoryHighWatermark),
|
||||
EvictingCPULowWatermark: utilpointer.Int(utils.DefaultEvictingCPULowWatermark),
|
||||
EvictingMemoryLowWatermark: utilpointer.Int(utils.DefaultEvictingMemoryLowWatermark),
|
||||
},
|
||||
},
|
||||
NodesConfig: []api.NodesConfig{
|
||||
{
|
||||
Selector: &metav1.LabelSelector{
|
||||
MatchLabels: map[string]string{
|
||||
"label-key": "label-value",
|
||||
},
|
||||
},
|
||||
ColocationConfig: api.ColocationConfig{EvictingConfig: &api.Evicting{
|
||||
EvictingCPUHighWatermark: utilpointer.Int(60),
|
||||
}},
|
||||
},
|
||||
}},
|
||||
},
|
||||
{
|
||||
name: "configmap-not-existed",
|
||||
initialObjects: []runtime.Object{},
|
||||
expectedErrIsNil: true,
|
||||
expectedConfig: defaultCfg(),
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
client := fake.NewSimpleClientset(tc.initialObjects...)
|
||||
mgr := &ConfigManager{
|
||||
kubeClient: client,
|
||||
configmapNamespace: "kube-system",
|
||||
configmapName: "volcano-agent-configuration",
|
||||
}
|
||||
actualErr := mgr.PrepareConfigmap()
|
||||
assert.Equal(t, tc.expectedErrIsNil, actualErr == nil)
|
||||
cm, err := client.CoreV1().ConfigMaps("kube-system").Get(context.TODO(), "volcano-agent-configuration", metav1.GetOptions{})
|
||||
assert.NoError(t, err)
|
||||
c := &api.VolcanoAgentConfig{}
|
||||
err = json.Unmarshal([]byte(cm.Data[utils.ColocationConfigKey]), c)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, tc.expectedConfig, c)
|
||||
})
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
type FakeListener struct {
|
||||
called int
|
||||
initial bool
|
||||
errors int
|
||||
}
|
||||
|
||||
func (l *FakeListener) SyncConfig(cfg *api.ColocationConfig) error {
|
||||
l.called++
|
||||
if !l.initial {
|
||||
l.initial = true
|
||||
return nil
|
||||
}
|
||||
if l.errors == 0 {
|
||||
return nil
|
||||
}
|
||||
l.errors--
|
||||
return errors.New("error")
|
||||
}
|
||||
|
||||
type FakeRateLimiter struct{}
|
||||
|
||||
func (r *FakeRateLimiter) When(item interface{}) time.Duration {
|
||||
return 0
|
||||
}
|
||||
|
||||
func (r *FakeRateLimiter) NumRequeues(item interface{}) int {
|
||||
return 0
|
||||
}
|
||||
|
||||
func (r *FakeRateLimiter) Forget(item interface{}) {}
|
||||
|
||||
func TestStart(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
initialObjects []runtime.Object
|
||||
sourceQueueObjects []interface{}
|
||||
ListerSyncErrTimes int
|
||||
expectedNotifyCalled int
|
||||
expectedErrIsNil bool
|
||||
}{
|
||||
{
|
||||
name: "without-sync-retry",
|
||||
initialObjects: []runtime.Object{makeConfigMap(utils.DefaultCfg)},
|
||||
sourceQueueObjects: []interface{}{"node", "configmap"},
|
||||
ListerSyncErrTimes: 0,
|
||||
expectedNotifyCalled: 3,
|
||||
expectedErrIsNil: true,
|
||||
},
|
||||
{
|
||||
name: "with-sync-retry",
|
||||
initialObjects: []runtime.Object{makeConfigMap(utils.DefaultCfg)},
|
||||
sourceQueueObjects: []interface{}{"node", "configmap"},
|
||||
ListerSyncErrTimes: 2,
|
||||
expectedNotifyCalled: 5,
|
||||
expectedErrIsNil: true,
|
||||
},
|
||||
}
|
||||
|
||||
fakeClient := fake.NewSimpleClientset(agentPod)
|
||||
informerFactory := informers.NewSharedInformerFactory(fakeClient, 0)
|
||||
informerFactory.Core().V1().Pods().Informer()
|
||||
informerFactory.Start(context.TODO().Done())
|
||||
cache.WaitForNamedCacheSync("", context.TODO().Done(), informerFactory.Core().V1().Pods().Informer().HasSynced)
|
||||
|
||||
for _, tc := range tests {
|
||||
l := &FakeListener{
|
||||
errors: tc.ListerSyncErrTimes,
|
||||
}
|
||||
queue := workqueue.NewNamedRateLimitingQueue(&FakeRateLimiter{}, "configmap-resource")
|
||||
for _, ob := range tc.sourceQueueObjects {
|
||||
queue.Add(ob)
|
||||
}
|
||||
s := source.NewFakeSource(queue)
|
||||
client := fake.NewSimpleClientset(tc.initialObjects...)
|
||||
mgr := &ConfigManager{
|
||||
kubeClient: client,
|
||||
configmapNamespace: "kube-system",
|
||||
configmapName: "volcano-agent-configuration",
|
||||
agentPodNamespace: "kube-system",
|
||||
agentPodName: "volcano-agent-pod",
|
||||
source: s,
|
||||
listeners: []Listener{l},
|
||||
podLister: informerFactory.Core().V1().Pods().Lister(),
|
||||
recorder: &record.FakeRecorder{},
|
||||
}
|
||||
actualErr := mgr.Start(context.TODO())
|
||||
s.Stop()
|
||||
assert.Equal(t, tc.expectedErrIsNil, actualErr == nil)
|
||||
assert.Equal(t, tc.expectedNotifyCalled, l.called)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,155 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package source
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"golang.org/x/time/rate"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/fields"
|
||||
"k8s.io/client-go/informers"
|
||||
coreinformers "k8s.io/client-go/informers/core/v1"
|
||||
clientset "k8s.io/client-go/kubernetes"
|
||||
"k8s.io/client-go/tools/cache"
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/config/utils"
|
||||
)
|
||||
|
||||
type configMapSource struct {
|
||||
kubeClient clientset.Interface
|
||||
informerFactory informers.SharedInformerFactory
|
||||
configMapListerSynced cache.InformerSynced
|
||||
nodeListerSynced cache.InformerSynced
|
||||
queue workqueue.RateLimitingInterface
|
||||
nodeName string
|
||||
configmapNamespace string
|
||||
configmapName string
|
||||
}
|
||||
|
||||
func NewConfigMapSource(client clientset.Interface, nodeName, namespace string) ConfigEventSource {
|
||||
tweakListOptions := func(options *metav1.ListOptions) {
|
||||
options.FieldSelector = fields.OneTermEqualSelector(utils.ObjectNameField, utils.ConfigMapName).String()
|
||||
}
|
||||
informerFactory := informers.NewSharedInformerFactory(client, 0)
|
||||
configMapInformer := informerFactory.InformerFor(&v1.ConfigMap{},
|
||||
func(client clientset.Interface, resyncPeriod time.Duration) cache.SharedIndexInformer {
|
||||
return coreinformers.NewFilteredConfigMapInformer(client, namespace, resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc}, tweakListOptions)
|
||||
})
|
||||
nodeInformer := informerFactory.InformerFor(&corev1.Node{},
|
||||
func(client clientset.Interface, resyncPeriod time.Duration) cache.SharedIndexInformer {
|
||||
tweakListOptions := func(options *metav1.ListOptions) {
|
||||
options.FieldSelector = fields.OneTermEqualSelector(metav1.ObjectNameField, nodeName).String()
|
||||
}
|
||||
return coreinformers.NewFilteredNodeInformer(client, resyncPeriod, cache.Indexers{}, tweakListOptions)
|
||||
})
|
||||
cs := &configMapSource{
|
||||
informerFactory: informerFactory,
|
||||
configMapListerSynced: configMapInformer.HasSynced,
|
||||
nodeListerSynced: nodeInformer.HasSynced,
|
||||
kubeClient: client,
|
||||
queue: workqueue.NewNamedRateLimitingQueue(workqueue.NewMaxOfRateLimiter(
|
||||
workqueue.NewItemExponentialFailureRateLimiter(500*time.Millisecond, 1000*time.Second),
|
||||
&workqueue.BucketRateLimiter{Limiter: rate.NewLimiter(rate.Limit(10), 100)}), "configmap-resource"),
|
||||
nodeName: nodeName,
|
||||
configmapNamespace: namespace,
|
||||
configmapName: utils.ConfigMapName,
|
||||
}
|
||||
configMapInformer.AddEventHandler(cache.ResourceEventHandlerFuncs{
|
||||
AddFunc: func(obj interface{}) { cs.queue.Add("configmap") },
|
||||
UpdateFunc: func(oldObj, newObj interface{}) { cs.queue.Add("configmap") },
|
||||
})
|
||||
nodeInformer.AddEventHandler(cache.ResourceEventHandlerFuncs{
|
||||
AddFunc: func(obj interface{}) { cs.nodeAdd(obj) },
|
||||
UpdateFunc: func(oldObj, newObj interface{}) { cs.nodeUpdate(oldObj, newObj) },
|
||||
})
|
||||
|
||||
return cs
|
||||
}
|
||||
|
||||
func (cs *configMapSource) nodeAdd(obj interface{}) {
|
||||
cs.queue.Add("node")
|
||||
}
|
||||
|
||||
func (cs *configMapSource) nodeUpdate(oldObj, newObj interface{}) {
|
||||
oldNode, ok := oldObj.(*corev1.Node)
|
||||
if !ok {
|
||||
klog.ErrorS(nil, "can not covert interface object to Node", "oldobj", oldObj)
|
||||
cs.queue.Add("node")
|
||||
return
|
||||
}
|
||||
|
||||
newNode, ok := newObj.(*corev1.Node)
|
||||
if !ok {
|
||||
klog.ErrorS(nil, "can not covert interface object to Node", "newobj", newObj)
|
||||
cs.queue.Add("node")
|
||||
return
|
||||
}
|
||||
|
||||
if oldNode.Labels[apis.ColocationEnableNodeLabelKey] == newNode.Labels[apis.ColocationEnableNodeLabelKey] &&
|
||||
oldNode.Labels[apis.OverSubscriptionNodeLabelKey] == newNode.Labels[apis.OverSubscriptionNodeLabelKey] &&
|
||||
oldNode.Labels[apis.ColocationPolicyKey] == newNode.Labels[apis.ColocationPolicyKey] {
|
||||
return
|
||||
}
|
||||
cs.queue.Add("node")
|
||||
}
|
||||
|
||||
func (cs *configMapSource) GetLatestConfig() (cfg *api.ColocationConfig, err error) {
|
||||
node, err := cs.informerFactory.Core().V1().Nodes().Lister().Get(cs.nodeName)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get node", "node", cs.nodeName)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
configmap, err := cs.informerFactory.Core().V1().ConfigMaps().Lister().ConfigMaps(cs.configmapNamespace).Get(cs.configmapName)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get configmap", "name", cs.configmapName)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
config := &api.VolcanoAgentConfig{}
|
||||
data, ok := configmap.Data[utils.ColocationConfigKey]
|
||||
if !ok || len(data) == 0 {
|
||||
return nil, fmt.Errorf("configMap data(%s) is empty", utils.ColocationConfigKey)
|
||||
}
|
||||
if err = json.Unmarshal([]byte(data), config); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return utils.MergerCfg(config, node)
|
||||
}
|
||||
|
||||
func (cs *configMapSource) Source(stopCh <-chan struct{}) (change workqueue.RateLimitingInterface, err error) {
|
||||
cs.informerFactory.Start(stopCh)
|
||||
if !cache.WaitForCacheSync(stopCh, cs.configMapListerSynced, cs.nodeListerSynced) {
|
||||
return nil, fmt.Errorf("timed out waiting for mapfigmap source caches to sync")
|
||||
}
|
||||
|
||||
go func() {
|
||||
<-stopCh
|
||||
cs.queue.ShutDown()
|
||||
}()
|
||||
|
||||
return cs.queue, nil
|
||||
}
|
||||
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package source
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
)
|
||||
|
||||
type fakeSource struct {
|
||||
queue workqueue.RateLimitingInterface
|
||||
cfg *api.ColocationConfig
|
||||
}
|
||||
|
||||
func NewFakeSource(queue workqueue.RateLimitingInterface) *fakeSource {
|
||||
return &fakeSource{
|
||||
queue: queue,
|
||||
}
|
||||
}
|
||||
|
||||
func (f *fakeSource) Source(stopCh <-chan struct{}) (change workqueue.RateLimitingInterface, err error) {
|
||||
return f.queue, nil
|
||||
}
|
||||
|
||||
func (f *fakeSource) GetLatestConfig() (cfg *api.ColocationConfig, err error) {
|
||||
return f.cfg, nil
|
||||
}
|
||||
|
||||
func (f *fakeSource) Stop() {
|
||||
for {
|
||||
if f.queue.Len() == 0 {
|
||||
f.queue.ShutDown()
|
||||
return
|
||||
}
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package source
|
||||
|
||||
import (
|
||||
"os"
|
||||
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
)
|
||||
|
||||
// TODO: implement file source.
|
||||
type fileSource struct {
|
||||
filePath string
|
||||
queue workqueue.RateLimitingInterface
|
||||
}
|
||||
|
||||
func NewFileSource(filePath string) ConfigEventSource {
|
||||
return &fileSource{}
|
||||
}
|
||||
|
||||
func (f *fileSource) Source(stopCh <-chan struct{}) (change workqueue.RateLimitingInterface, err error) {
|
||||
return f.queue, nil
|
||||
}
|
||||
|
||||
func (f *fileSource) GetLatestConfig() (cfg *api.ColocationConfig, err error) {
|
||||
_, err = os.ReadFile(f.filePath)
|
||||
return nil, err
|
||||
}
|
||||
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package source
|
||||
|
||||
import (
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
)
|
||||
|
||||
// ConfigEventSource is the source of config.
|
||||
type ConfigEventSource interface {
|
||||
// Source returns the queue with rateLimited,
|
||||
Source(stopCh <-chan struct{}) (change workqueue.RateLimitingInterface, err error)
|
||||
// GetLatestConfig returns the latest colocation configuration from source
|
||||
GetLatestConfig() (cfg *api.ColocationConfig, err error)
|
||||
}
|
||||
|
|
@ -0,0 +1,178 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package utils
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
|
||||
"github.com/imdario/mergo"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/labels"
|
||||
utilerrors "k8s.io/apimachinery/pkg/util/errors"
|
||||
"k8s.io/klog/v2"
|
||||
utilpointer "k8s.io/utils/pointer"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
)
|
||||
|
||||
const (
|
||||
ConfigMapName = "volcano-agent-configuration"
|
||||
ColocationConfigKey = "colocation-config"
|
||||
ObjectNameField = "metadata.name"
|
||||
)
|
||||
|
||||
const (
|
||||
// Network Qos config
|
||||
DefaultOnlineBandwidthWatermarkPercent = 80
|
||||
DefaultOfflineLowBandwidthPercent = 10
|
||||
DefaultOfflineHighBandwidthPercent = 40
|
||||
DefaultNetworkQoSInterval = 10000000 // 1000000 纳秒 = 10 毫秒
|
||||
|
||||
// OverSubscription config
|
||||
DefaultOverSubscriptionTypes = "cpu,memory"
|
||||
|
||||
// Evicting config
|
||||
DefaultEvictingCPUHighWatermark = 80
|
||||
DefaultEvictingMemoryHighWatermark = 60
|
||||
DefaultEvictingCPULowWatermark = 30
|
||||
DefaultEvictingMemoryLowWatermark = 30
|
||||
)
|
||||
|
||||
const (
|
||||
DefaultCfg = `
|
||||
{
|
||||
"globalConfig":{
|
||||
"cpuBurstConfig":{
|
||||
"enable":true
|
||||
},
|
||||
"networkQosConfig":{
|
||||
"enable":true,
|
||||
"onlineBandwidthWatermarkPercent":80,
|
||||
"offlineLowBandwidthPercent":10,
|
||||
"offlineHighBandwidthPercent":40,
|
||||
"qosCheckInterval": 10000000
|
||||
},
|
||||
"overSubscriptionConfig":{
|
||||
"enable":true,
|
||||
"overSubscriptionTypes":"cpu,memory"
|
||||
},
|
||||
"evictingConfig":{
|
||||
"evictingCPUHighWatermark":80,
|
||||
"evictingMemoryHighWatermark":60,
|
||||
"evictingCPULowWatermark":30,
|
||||
"evictingMemoryLowWatermark":30
|
||||
}
|
||||
}
|
||||
}
|
||||
`
|
||||
)
|
||||
|
||||
// DefaultColocationConfig is the default colocation config.
|
||||
func DefaultColocationConfig() *api.ColocationConfig {
|
||||
return &api.ColocationConfig{
|
||||
NodeLabelConfig: &api.NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(false),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(false),
|
||||
},
|
||||
CPUQosConfig: &api.CPUQos{Enable: utilpointer.Bool(true)},
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(true)},
|
||||
MemoryQosConfig: &api.MemoryQos{Enable: utilpointer.Bool(true)},
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(DefaultOnlineBandwidthWatermarkPercent),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(DefaultOfflineLowBandwidthPercent),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(DefaultOfflineHighBandwidthPercent),
|
||||
QoSCheckInterval: utilpointer.Int(DefaultNetworkQoSInterval),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String(DefaultOverSubscriptionTypes),
|
||||
},
|
||||
EvictingConfig: &api.Evicting{
|
||||
EvictingCPUHighWatermark: utilpointer.Int(DefaultEvictingCPUHighWatermark),
|
||||
EvictingMemoryHighWatermark: utilpointer.Int(DefaultEvictingMemoryHighWatermark),
|
||||
EvictingCPULowWatermark: utilpointer.Int(DefaultEvictingCPULowWatermark),
|
||||
EvictingMemoryLowWatermark: utilpointer.Int(DefaultEvictingMemoryLowWatermark),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// DefaultVolcanoAgentConfig returns default volcano agent config.
|
||||
func DefaultVolcanoAgentConfig() *api.VolcanoAgentConfig {
|
||||
return &api.VolcanoAgentConfig{
|
||||
GlobalConfig: DefaultColocationConfig(),
|
||||
}
|
||||
}
|
||||
|
||||
type nullTransformer struct {
|
||||
}
|
||||
|
||||
// Transformer temporary solution, waiting https://github.com/imdario/mergo/issues/131 to be fixed.
|
||||
func (t *nullTransformer) Transformer(typ reflect.Type) func(dst, src reflect.Value) error {
|
||||
if typ.Kind() == reflect.Pointer && typ.Elem().Kind() != reflect.Struct {
|
||||
return func(dst, src reflect.Value) error {
|
||||
if dst.CanSet() && !src.IsNil() {
|
||||
dst.Set(src)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func MergerCfg(fullConfig *api.VolcanoAgentConfig, node *corev1.Node) (*api.ColocationConfig, error) {
|
||||
mergedCfg := DefaultColocationConfig()
|
||||
defaultCfg := DefaultColocationConfig()
|
||||
if fullConfig == nil || fullConfig.GlobalConfig == nil {
|
||||
klog.InfoS("full or global config is nil, use default config")
|
||||
return defaultCfg, nil
|
||||
}
|
||||
|
||||
if err := mergo.Merge(mergedCfg, fullConfig.GlobalConfig, mergo.WithOverride, mergo.WithTransformers(&nullTransformer{})); err != nil {
|
||||
klog.ErrorS(err, "Failed to merge default and global config, use default config")
|
||||
return defaultCfg, nil
|
||||
}
|
||||
|
||||
nodeCfg := &api.ColocationConfig{}
|
||||
for idx := range fullConfig.NodesConfig {
|
||||
selector, err := metav1.LabelSelectorAsSelector(fullConfig.NodesConfig[idx].Selector)
|
||||
if err != nil || !selector.Matches(labels.Set(node.Labels)) {
|
||||
continue
|
||||
}
|
||||
// choose the last config if multi labels matched.
|
||||
nodeCfg = &fullConfig.NodesConfig[idx].ColocationConfig
|
||||
}
|
||||
|
||||
if err := mergo.Merge(mergedCfg, nodeCfg, mergo.WithOverride, mergo.WithTransformers(&nullTransformer{})); err != nil {
|
||||
klog.ErrorS(err, "Failed to merge node config")
|
||||
return mergedCfg, err
|
||||
}
|
||||
|
||||
enableOverSubscription := utilpointer.Bool(utilnode.IsNodeSupportOverSubscription(node))
|
||||
mergedCfg.NodeLabelConfig.NodeColocationEnable = utilpointer.Bool(utilnode.IsNodeSupportColocation(node) || *enableOverSubscription)
|
||||
mergedCfg.NodeLabelConfig.NodeOverSubscriptionEnable = enableOverSubscription
|
||||
|
||||
validateErr := utilerrors.NewAggregate(mergedCfg.Validate())
|
||||
if validateErr != nil {
|
||||
klog.ErrorS(validateErr, "Config is invalid, keep original config")
|
||||
return mergedCfg, validateErr
|
||||
}
|
||||
|
||||
return mergedCfg, nil
|
||||
}
|
||||
|
|
@ -0,0 +1,255 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package utils
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
utilpointer "k8s.io/utils/pointer"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
)
|
||||
|
||||
func TestMergerCfg(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
volcanoCfg *api.VolcanoAgentConfig
|
||||
node *corev1.Node
|
||||
wantCfg *api.ColocationConfig
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
name: "disable cpu qos && disable cpu burst",
|
||||
volcanoCfg: &api.VolcanoAgentConfig{
|
||||
GlobalConfig: &api.ColocationConfig{
|
||||
CPUQosConfig: &api.CPUQos{Enable: utilpointer.Bool(false)},
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(false)},
|
||||
},
|
||||
},
|
||||
node: &corev1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{},
|
||||
},
|
||||
wantCfg: disableCPUBurst(disableCPUBurst(disableCPUQos(DefaultColocationConfig()))),
|
||||
wantErr: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "disable memory qos && disable overSubscription",
|
||||
volcanoCfg: &api.VolcanoAgentConfig{
|
||||
GlobalConfig: &api.ColocationConfig{
|
||||
MemoryQosConfig: &api.MemoryQos{Enable: utilpointer.Bool(false)},
|
||||
OverSubscriptionConfig: &api.OverSubscription{Enable: utilpointer.Bool(false)},
|
||||
},
|
||||
},
|
||||
node: &corev1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Labels: map[string]string{
|
||||
apis.ColocationEnableNodeLabelKey: "true",
|
||||
apis.OverSubscriptionNodeLabelKey: "true",
|
||||
},
|
||||
},
|
||||
},
|
||||
wantCfg: disableOverSubscription(disableMemoryQos(enableNodeOverSubscription(enableNodeColocation(DefaultColocationConfig())))),
|
||||
wantErr: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "disable network qos",
|
||||
volcanoCfg: &api.VolcanoAgentConfig{
|
||||
GlobalConfig: &api.ColocationConfig{
|
||||
NetworkQosConfig: &api.NetworkQos{Enable: utilpointer.Bool(false)},
|
||||
},
|
||||
},
|
||||
node: &corev1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Labels: map[string]string{
|
||||
apis.ColocationEnableNodeLabelKey: "true",
|
||||
},
|
||||
},
|
||||
},
|
||||
wantCfg: disableNetworkQos(enableNodeColocation(DefaultColocationConfig())),
|
||||
wantErr: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "modify network qos",
|
||||
volcanoCfg: &api.VolcanoAgentConfig{
|
||||
GlobalConfig: &api.ColocationConfig{
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(15),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(16),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(17),
|
||||
},
|
||||
},
|
||||
},
|
||||
node: &corev1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Labels: map[string]string{
|
||||
apis.OverSubscriptionNodeLabelKey: "true",
|
||||
},
|
||||
},
|
||||
},
|
||||
wantCfg: withNewNetworkQoS(enableNodeOverSubscription(enableNodeColocation(DefaultColocationConfig())), &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
QoSCheckInterval: utilpointer.Int(10000000),
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(15),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(16),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(17),
|
||||
}),
|
||||
wantErr: false,
|
||||
},
|
||||
|
||||
{
|
||||
name: "modify network qos && OnlineBandwidthWatermarkPercent=0",
|
||||
volcanoCfg: &api.VolcanoAgentConfig{
|
||||
GlobalConfig: &api.ColocationConfig{
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(0),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(18),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(19),
|
||||
},
|
||||
},
|
||||
},
|
||||
node: &corev1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Labels: map[string]string{
|
||||
apis.OverSubscriptionNodeLabelKey: "true",
|
||||
},
|
||||
},
|
||||
},
|
||||
wantCfg: withNewNetworkQoS(enableNodeOverSubscription(enableNodeColocation(DefaultColocationConfig())), &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
QoSCheckInterval: utilpointer.Int(10000000),
|
||||
OnlineBandwidthWatermarkPercent: utilpointer.Int(0),
|
||||
OfflineLowBandwidthPercent: utilpointer.Int(18),
|
||||
OfflineHighBandwidthPercent: utilpointer.Int(19),
|
||||
}),
|
||||
wantErr: true,
|
||||
},
|
||||
|
||||
{
|
||||
name: "modify OverSubscription && OverSubscriptionTypes=\"\"",
|
||||
volcanoCfg: &api.VolcanoAgentConfig{
|
||||
GlobalConfig: &api.ColocationConfig{
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String(""),
|
||||
},
|
||||
},
|
||||
},
|
||||
node: &corev1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Labels: map[string]string{
|
||||
apis.OverSubscriptionNodeLabelKey: "true",
|
||||
},
|
||||
},
|
||||
},
|
||||
wantCfg: withNewOverSubscription(enableNodeOverSubscription(enableNodeColocation(DefaultColocationConfig())), &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String(""),
|
||||
}),
|
||||
wantErr: false,
|
||||
},
|
||||
{
|
||||
name: "overwrite default config label selector",
|
||||
volcanoCfg: &api.VolcanoAgentConfig{
|
||||
GlobalConfig: DefaultColocationConfig(),
|
||||
NodesConfig: []api.NodesConfig{
|
||||
{
|
||||
Selector: &metav1.LabelSelector{MatchLabels: map[string]string{
|
||||
"label-key": "label-value",
|
||||
}},
|
||||
ColocationConfig: api.ColocationConfig{EvictingConfig: &api.Evicting{
|
||||
EvictingCPUHighWatermark: utilpointer.Int(10),
|
||||
}},
|
||||
},
|
||||
},
|
||||
},
|
||||
node: &corev1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Labels: map[string]string{
|
||||
"label-key": "label-value",
|
||||
},
|
||||
},
|
||||
},
|
||||
wantCfg: withLabelSelector(DefaultColocationConfig()),
|
||||
wantErr: true,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range tests {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
actualCfg, actualErr := MergerCfg(tc.volcanoCfg, tc.node)
|
||||
assert.Equal(t, tc.wantCfg, actualCfg, tc.name)
|
||||
assert.Equal(t, tc.wantErr, actualErr != nil, tc.name)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func withNewOverSubscription(config *api.ColocationConfig, overSubscription *api.OverSubscription) *api.ColocationConfig {
|
||||
config.OverSubscriptionConfig = overSubscription
|
||||
return config
|
||||
}
|
||||
|
||||
func withNewNetworkQoS(config *api.ColocationConfig, networkqos *api.NetworkQos) *api.ColocationConfig {
|
||||
config.NetworkQosConfig = networkqos
|
||||
return config
|
||||
}
|
||||
|
||||
func enableNodeColocation(config *api.ColocationConfig) *api.ColocationConfig {
|
||||
config.NodeLabelConfig.NodeColocationEnable = utilpointer.Bool(true)
|
||||
return config
|
||||
}
|
||||
|
||||
func enableNodeOverSubscription(config *api.ColocationConfig) *api.ColocationConfig {
|
||||
config.NodeLabelConfig.NodeOverSubscriptionEnable = utilpointer.Bool(true)
|
||||
return config
|
||||
}
|
||||
|
||||
func disableCPUQos(config *api.ColocationConfig) *api.ColocationConfig {
|
||||
config.CPUQosConfig.Enable = utilpointer.Bool(false)
|
||||
return config
|
||||
}
|
||||
|
||||
func disableOverSubscription(config *api.ColocationConfig) *api.ColocationConfig {
|
||||
config.OverSubscriptionConfig.Enable = utilpointer.Bool(false)
|
||||
return config
|
||||
}
|
||||
|
||||
func disableNetworkQos(config *api.ColocationConfig) *api.ColocationConfig {
|
||||
config.NetworkQosConfig.Enable = utilpointer.Bool(false)
|
||||
return config
|
||||
}
|
||||
|
||||
func disableMemoryQos(config *api.ColocationConfig) *api.ColocationConfig {
|
||||
config.MemoryQosConfig.Enable = utilpointer.Bool(false)
|
||||
return config
|
||||
}
|
||||
|
||||
func disableCPUBurst(config *api.ColocationConfig) *api.ColocationConfig {
|
||||
config.CPUBurstConfig.Enable = utilpointer.Bool(false)
|
||||
return config
|
||||
}
|
||||
|
||||
func withLabelSelector(config *api.ColocationConfig) *api.ColocationConfig {
|
||||
config.EvictingConfig.EvictingCPUHighWatermark = utilpointer.Int(10)
|
||||
return config
|
||||
}
|
||||
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package events
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
coloconfig "volcano.sh/volcano/pkg/agent/config"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers"
|
||||
"volcano.sh/volcano/pkg/agent/events/probes"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
|
||||
_ "volcano.sh/volcano/pkg/agent/events/handlers/cpuburst"
|
||||
_ "volcano.sh/volcano/pkg/agent/events/handlers/cpuqos"
|
||||
_ "volcano.sh/volcano/pkg/agent/events/handlers/eviction"
|
||||
_ "volcano.sh/volcano/pkg/agent/events/handlers/memoryqos"
|
||||
_ "volcano.sh/volcano/pkg/agent/events/handlers/networkqos"
|
||||
_ "volcano.sh/volcano/pkg/agent/events/handlers/oversubscription"
|
||||
_ "volcano.sh/volcano/pkg/agent/events/handlers/resources"
|
||||
_ "volcano.sh/volcano/pkg/agent/events/probes/nodemonitor"
|
||||
_ "volcano.sh/volcano/pkg/agent/events/probes/noderesources"
|
||||
_ "volcano.sh/volcano/pkg/agent/events/probes/pods"
|
||||
)
|
||||
|
||||
type EventManager struct {
|
||||
eventQueueFactory *framework.EventQueueFactory
|
||||
config *config.Configuration
|
||||
metricCollectManager *metriccollect.MetricCollectorManager
|
||||
configMgr *coloconfig.ConfigManager
|
||||
}
|
||||
|
||||
func NewEventManager(config *config.Configuration, metricCollectManager *metriccollect.MetricCollectorManager, cgroupMgr cgroup.CgroupManager) *EventManager {
|
||||
factory := &framework.EventQueueFactory{}
|
||||
factory.Queues = make(map[string]*framework.EventQueue)
|
||||
mgr := &EventManager{
|
||||
config: config,
|
||||
metricCollectManager: metricCollectManager,
|
||||
eventQueueFactory: factory,
|
||||
configMgr: coloconfig.NewManager(config, []coloconfig.Listener{factory}),
|
||||
}
|
||||
|
||||
for eventName, newProbeFuncs := range probes.GetEventProbeFuncs() {
|
||||
eventQueue := mgr.eventQueueFactory.EventQueue(eventName)
|
||||
for _, newProbeFunc := range newProbeFuncs {
|
||||
prob := newProbeFunc(config, metricCollectManager, eventQueue.GetQueue())
|
||||
mgr.eventQueueFactory.RegistryEventProbe(eventName, prob)
|
||||
}
|
||||
}
|
||||
|
||||
for eventName, newHandleFuncs := range handlers.GetEventHandlerFuncs() {
|
||||
for _, newHandleFunc := range newHandleFuncs {
|
||||
handle := newHandleFunc(config, metricCollectManager, cgroupMgr)
|
||||
mgr.eventQueueFactory.RegistryEventHandler(eventName, handle)
|
||||
}
|
||||
}
|
||||
return mgr
|
||||
}
|
||||
|
||||
func (m *EventManager) Run(ctx context.Context) error {
|
||||
klog.InfoS("Start event manager")
|
||||
if err := m.configMgr.Start(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := m.eventQueueFactory.Start(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
klog.InfoS("Successfully started event manager")
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,106 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package framework
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"golang.org/x/time/rate"
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
"k8s.io/klog/v2"
|
||||
)
|
||||
|
||||
type EventQueue struct {
|
||||
// Name is the name of event.
|
||||
Name string
|
||||
// Workers is the number of handle workers that are allowed to sync concurrently.
|
||||
Workers int
|
||||
// Queue for caching events
|
||||
Queue workqueue.RateLimitingInterface
|
||||
// List of handlers that need to handle this event.
|
||||
Handlers []Handle
|
||||
// List of event detectors.
|
||||
// Generate an event and put the event in the queue
|
||||
Probes []Probe
|
||||
}
|
||||
|
||||
func NewEventQueue(name string) *EventQueue {
|
||||
rateLimiter := workqueue.NewMaxOfRateLimiter(
|
||||
workqueue.NewItemExponentialFailureRateLimiter(1*time.Second, 100*time.Second),
|
||||
&workqueue.BucketRateLimiter{Limiter: rate.NewLimiter(rate.Limit(10), 100)},
|
||||
)
|
||||
return &EventQueue{
|
||||
Name: name,
|
||||
Workers: 30,
|
||||
Queue: workqueue.NewNamedRateLimitingQueue(rateLimiter, name),
|
||||
}
|
||||
}
|
||||
|
||||
func (eq *EventQueue) SetWorkers(workers int) {
|
||||
eq.Workers = workers
|
||||
}
|
||||
|
||||
func (eq *EventQueue) AddHandler(handle ...Handle) {
|
||||
eq.Handlers = append(eq.Handlers, handle...)
|
||||
}
|
||||
|
||||
func (eq *EventQueue) AddProbe(p ...Probe) {
|
||||
eq.Probes = append(eq.Probes, p...)
|
||||
}
|
||||
|
||||
func (eq *EventQueue) GetQueue() workqueue.RateLimitingInterface {
|
||||
return eq.Queue
|
||||
}
|
||||
|
||||
func (eq *EventQueue) ProcessEvent(ctx context.Context) {
|
||||
for {
|
||||
if !eq.processNextWorkItem(ctx) {
|
||||
klog.ErrorS(nil, "Event queue shut down")
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (eq *EventQueue) processNextWorkItem(ctx context.Context) bool {
|
||||
key, quit := eq.Queue.Get()
|
||||
if quit {
|
||||
return false
|
||||
}
|
||||
defer eq.Queue.Done(key)
|
||||
|
||||
visitErr := false
|
||||
defer func() {
|
||||
if visitErr {
|
||||
eq.Queue.AddRateLimited(key)
|
||||
} else {
|
||||
eq.Queue.Forget(key)
|
||||
}
|
||||
}()
|
||||
|
||||
for _, handler := range eq.Handlers {
|
||||
if !handler.IsActive() {
|
||||
continue
|
||||
}
|
||||
err := handler.Handle(key)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Handle process failed", "handler", handler.HandleName())
|
||||
visitErr = true
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package framework
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
utilerrors "k8s.io/apimachinery/pkg/util/errors"
|
||||
"k8s.io/apimachinery/pkg/util/wait"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
)
|
||||
|
||||
type EventQueueFactory struct {
|
||||
Mutex sync.RWMutex
|
||||
Queues map[string]*EventQueue
|
||||
}
|
||||
|
||||
func (f *EventQueueFactory) EventQueue(name string) *EventQueue {
|
||||
f.Mutex.Lock()
|
||||
defer f.Mutex.Unlock()
|
||||
|
||||
if queue, ok := f.Queues[name]; ok {
|
||||
return queue
|
||||
}
|
||||
|
||||
eventQueue := NewEventQueue(name)
|
||||
f.Queues[name] = eventQueue
|
||||
return eventQueue
|
||||
}
|
||||
|
||||
func (f *EventQueueFactory) RegistryEventHandler(name string, handle Handle) Handle {
|
||||
f.EventQueue(name).AddHandler(handle)
|
||||
klog.InfoS("Successfully registry event handler", "event", name, "handler", handle.HandleName())
|
||||
return handle
|
||||
}
|
||||
|
||||
func (f *EventQueueFactory) RegistryEventProbe(name string, probe Probe) Probe {
|
||||
f.EventQueue(name).AddProbe(probe)
|
||||
klog.InfoS("Successfully registry event probe", "name", name, "probe", probe.ProbeName())
|
||||
return probe
|
||||
}
|
||||
|
||||
func (f *EventQueueFactory) SyncConfig(cfg *api.ColocationConfig) error {
|
||||
f.Mutex.Lock()
|
||||
defer f.Mutex.Unlock()
|
||||
|
||||
var errs []error
|
||||
for _, queue := range f.Queues {
|
||||
for _, handler := range queue.Handlers {
|
||||
if syncErr := handler.RefreshCfg(cfg); syncErr != nil {
|
||||
errs = append(errs, syncErr)
|
||||
}
|
||||
}
|
||||
for _, probe := range queue.Probes {
|
||||
if syncErr := probe.RefreshCfg(cfg); syncErr != nil {
|
||||
errs = append(errs, syncErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
return utilerrors.NewAggregate(errs)
|
||||
}
|
||||
|
||||
func (f *EventQueueFactory) Start(ctx context.Context) error {
|
||||
klog.InfoS("Start event queue factory")
|
||||
for _, event := range f.Queues {
|
||||
for _, probe := range event.Probes {
|
||||
go probe.Run(ctx.Done())
|
||||
}
|
||||
}
|
||||
|
||||
for _, event := range f.Queues {
|
||||
for i := 0; i < event.Workers; i++ {
|
||||
go wait.UntilWithContext(ctx, event.ProcessEvent, time.Second)
|
||||
}
|
||||
}
|
||||
klog.InfoS("Successfully started event queue factory")
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package framework
|
||||
|
||||
import (
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
)
|
||||
|
||||
type Probe interface {
|
||||
// ProbeName returns name of the probe
|
||||
ProbeName() string
|
||||
// Run runs the probe
|
||||
Run(stop <-chan struct{})
|
||||
// RefreshCfg hot update probe's cfg.
|
||||
RefreshCfg(cfg *api.ColocationConfig) error
|
||||
}
|
||||
|
||||
type Handle interface {
|
||||
// HandleName returns name of the handler
|
||||
HandleName() string
|
||||
// Handle handles the given event
|
||||
// Return an error only if the event needs to be re-enqueued to be processed
|
||||
// Need to avoid returning errors that cannot be resolved by retrying
|
||||
Handle(event interface{}) error
|
||||
// IsActive returns true if the handler is enabled
|
||||
IsActive() bool
|
||||
// RefreshCfg hot update handler's cfg.
|
||||
RefreshCfg(cfg *api.ColocationConfig) error
|
||||
}
|
||||
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package framework
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/types"
|
||||
)
|
||||
|
||||
type EventName string
|
||||
|
||||
const (
|
||||
PodEventName EventName = "PodsSync"
|
||||
|
||||
NodeResourcesEventName EventName = "NodeResourcesSync"
|
||||
|
||||
NodeMonitorEventName EventName = "NodeUtilizationSync"
|
||||
)
|
||||
|
||||
type PodEvent struct {
|
||||
UID types.UID
|
||||
QoSLevel int64
|
||||
QoSClass corev1.PodQOSClass
|
||||
Pod *corev1.Pod
|
||||
}
|
||||
|
||||
// NodeResourceEvent defines node resource event, overSubscription resource recently.
|
||||
type NodeResourceEvent struct {
|
||||
MillCPU int64
|
||||
MemoryBytes int64
|
||||
}
|
||||
|
||||
// NodeMonitorEvent defines node pressure event.
|
||||
type NodeMonitorEvent struct {
|
||||
// TimeStamp is the time when event occur.
|
||||
TimeStamp time.Time
|
||||
// Resource represents which resource is under pressure.
|
||||
Resource corev1.ResourceName
|
||||
}
|
||||
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package base
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
)
|
||||
|
||||
type BaseHandle struct {
|
||||
Name string
|
||||
Lock sync.RWMutex
|
||||
Config *config.Configuration
|
||||
Active bool
|
||||
}
|
||||
|
||||
func (h *BaseHandle) HandleName() string {
|
||||
return h.Name
|
||||
}
|
||||
|
||||
func (h *BaseHandle) IsActive() bool {
|
||||
h.Lock.Lock()
|
||||
defer h.Lock.Unlock()
|
||||
return h.Active
|
||||
}
|
||||
|
||||
func (h *BaseHandle) Handle(event interface{}) error {
|
||||
return errors.New("unimplemented")
|
||||
}
|
||||
|
||||
func (h *BaseHandle) RefreshCfg(cfg *api.ColocationConfig) error {
|
||||
h.Lock.Lock()
|
||||
defer h.Lock.Unlock()
|
||||
|
||||
isActive, err := features.DefaultFeatureGate.Enabled(features.Feature(h.HandleName()), cfg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if isActive {
|
||||
if supportErr := features.DefaultFeatureGate.Supported(features.Feature(h.HandleName()), h.Config); supportErr != nil {
|
||||
return supportErr
|
||||
}
|
||||
}
|
||||
|
||||
if h.Active != isActive {
|
||||
klog.InfoS("Event handler config changes", "handle", h.HandleName(), "nodeColocation", *cfg.NodeLabelConfig.NodeColocationEnable,
|
||||
"nodeOverSubscription", *cfg.NodeLabelConfig.NodeOverSubscriptionEnable, "old", h.Active, "new", isActive)
|
||||
}
|
||||
h.Active = isActive
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,177 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package cpuburst
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"io/fs"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strconv"
|
||||
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
v1 "k8s.io/client-go/informers/core/v1"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers/base"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/agent/utils"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/agent/utils/file"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
func init() {
|
||||
handlers.RegisterEventHandleFunc(string(framework.PodEventName), NewCPUBurst)
|
||||
}
|
||||
|
||||
type CPUBurstHandle struct {
|
||||
*base.BaseHandle
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
podInformer v1.PodInformer
|
||||
}
|
||||
|
||||
func NewCPUBurst(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, cgroupMgr cgroup.CgroupManager) framework.Handle {
|
||||
return &CPUBurstHandle{
|
||||
BaseHandle: &base.BaseHandle{
|
||||
Name: string(features.CPUBurstFeature),
|
||||
Config: config,
|
||||
},
|
||||
cgroupMgr: cgroupMgr,
|
||||
podInformer: config.InformerFactory.K8SInformerFactory.Core().V1().Pods(),
|
||||
}
|
||||
}
|
||||
|
||||
func (c *CPUBurstHandle) Handle(event interface{}) error {
|
||||
podEvent, ok := event.(framework.PodEvent)
|
||||
if !ok {
|
||||
return fmt.Errorf("illegal pod event")
|
||||
}
|
||||
pod := podEvent.Pod
|
||||
latestPod, err := c.podInformer.Lister().Pods(pod.Namespace).Get(pod.Name)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get pod from lister")
|
||||
} else {
|
||||
pod = latestPod
|
||||
}
|
||||
str, exists := pod.Annotations[EnabledKey]
|
||||
if !exists {
|
||||
return nil
|
||||
}
|
||||
enable, err := strconv.ParseBool(str)
|
||||
if err != nil || !enable {
|
||||
return nil
|
||||
}
|
||||
|
||||
cgroupPath, err := c.cgroupMgr.GetPodCgroupPath(podEvent.QoSClass, cgroup.CgroupCpuSubsystem, podEvent.UID)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to get pod cgroup file(%s), error: %v", podEvent.UID, err)
|
||||
}
|
||||
|
||||
quotaBurstTime := getCPUBurstTime(pod)
|
||||
podBurstTime := int64(0)
|
||||
err = filepath.WalkDir(cgroupPath, walkFunc(cgroupPath, quotaBurstTime, &podBurstTime))
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to set container cpu quota burst time, err: %v", err)
|
||||
}
|
||||
|
||||
// last set pod cgroup cpu quota burst.
|
||||
podQuotaTotalFile := filepath.Join(cgroupPath, cgroup.CPUQuotaTotalFile)
|
||||
value, err := file.ReadIntFromFile(podQuotaTotalFile)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to get pod cpu total quota time, err: %v,path: %s", err, podQuotaTotalFile)
|
||||
}
|
||||
if value == fixedQuotaValue {
|
||||
return nil
|
||||
}
|
||||
podQuotaBurstFile := filepath.Join(cgroupPath, cgroup.CPUQuotaBurstFile)
|
||||
err = utils.UpdateFile(podQuotaBurstFile, []byte(strconv.FormatInt(podBurstTime, 10)))
|
||||
if err != nil {
|
||||
if errors.Is(err, os.ErrNotExist) {
|
||||
klog.ErrorS(nil, "CPU Burst is not supported", "cgroupFile", podQuotaBurstFile)
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
klog.InfoS("Successfully set pod cpu quota burst time", "path", podQuotaBurstFile, "quotaBurst", podBurstTime, "pod", klog.KObj(pod))
|
||||
return nil
|
||||
}
|
||||
|
||||
func walkFunc(cgroupPath string, quotaBurstTime int64, podBurstTime *int64) fs.WalkDirFunc {
|
||||
return func(path string, d fs.DirEntry, err error) error {
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// We will set pod cgroup later.
|
||||
if path == cgroupPath {
|
||||
return nil
|
||||
}
|
||||
if d == nil || !d.IsDir() {
|
||||
return nil
|
||||
}
|
||||
quotaTotalFile := filepath.Join(path, cgroup.CPUQuotaTotalFile)
|
||||
quotaTotal, err := file.ReadIntFromFile(quotaTotalFile)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to get container cpu total quota time, err: %v, path: %s", err, quotaTotalFile)
|
||||
}
|
||||
if quotaTotal == fixedQuotaValue {
|
||||
return nil
|
||||
}
|
||||
|
||||
actualBurst := quotaBurstTime
|
||||
if quotaBurstTime > quotaTotal {
|
||||
klog.ErrorS(nil, "The quota burst time is greater than quota total, use quota total as burst time", "quotaBurst", quotaBurstTime, "quoTotal", quotaTotal)
|
||||
actualBurst = quotaTotal
|
||||
}
|
||||
if quotaBurstTime == 0 {
|
||||
actualBurst = quotaTotal
|
||||
}
|
||||
*podBurstTime += actualBurst
|
||||
quotaBurstFile := filepath.Join(path, cgroup.CPUQuotaBurstFile)
|
||||
err = utils.UpdateFile(quotaBurstFile, []byte(strconv.FormatInt(actualBurst, 10)))
|
||||
if err != nil {
|
||||
if errors.Is(err, os.ErrNotExist) {
|
||||
klog.ErrorS(nil, "CPU Burst is not supported", "cgroupFile", quotaBurstFile)
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
klog.InfoS("Successfully set container cpu burst time", "path", quotaBurstFile, "quotaTotal", quotaTotal, "quotaBurst", actualBurst)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func getCPUBurstTime(pod *corev1.Pod) int64 {
|
||||
var quotaBurstTime int64
|
||||
str, exists := pod.Annotations[QuotaTimeKey]
|
||||
if !exists {
|
||||
return quotaBurstTime
|
||||
}
|
||||
value, err := strconv.ParseInt(str, 10, 64)
|
||||
if err != nil || value <= 0 {
|
||||
klog.ErrorS(err, "Invalid quota burst time, use default containers' quota time", "value", str)
|
||||
return quotaBurstTime
|
||||
}
|
||||
quotaBurstTime = int64(value)
|
||||
return quotaBurstTime
|
||||
}
|
||||
|
|
@ -0,0 +1,264 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package cpuburst
|
||||
|
||||
import (
|
||||
"os"
|
||||
"path"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/client-go/informers"
|
||||
"k8s.io/client-go/kubernetes/fake"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/agent/utils/file"
|
||||
)
|
||||
|
||||
func TestCPUBurstHandle_Handle(t *testing.T) {
|
||||
tmpDir := t.TempDir()
|
||||
tests := []struct {
|
||||
name string
|
||||
event interface{}
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
prepare func()
|
||||
post func() map[string]string
|
||||
wantErr bool
|
||||
wantVal map[string]string
|
||||
}{
|
||||
{
|
||||
name: "not support cpu burst, return no err",
|
||||
event: framework.PodEvent{
|
||||
UID: "fake-id1",
|
||||
QoSLevel: 0,
|
||||
QoSClass: "",
|
||||
Pod: getPod("100000", "true"),
|
||||
},
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
|
||||
prepare: func() {
|
||||
prepare(t, tmpDir, "fake-id1", []info{
|
||||
{path: cgroup.CPUQuotaTotalFile, value: "100000"},
|
||||
})
|
||||
},
|
||||
wantErr: false,
|
||||
},
|
||||
{
|
||||
name: "quota=-1, no need set, return no err",
|
||||
event: framework.PodEvent{
|
||||
UID: "fake-id2",
|
||||
QoSLevel: 0,
|
||||
QoSClass: "",
|
||||
Pod: getPod("100000", "true"),
|
||||
},
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
prepare: func() {
|
||||
prepare(t, tmpDir, "fake-id2", []info{
|
||||
{path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{path: cgroup.CPUQuotaTotalFile, value: "-1"}})
|
||||
},
|
||||
wantErr: false,
|
||||
},
|
||||
{
|
||||
name: "one container quota=100000, another quota=-1, set quota burst successfully",
|
||||
event: framework.PodEvent{
|
||||
UID: "fake-id3",
|
||||
QoSLevel: 0,
|
||||
QoSClass: "",
|
||||
Pod: getPod("50000", "true"),
|
||||
},
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
prepare: func() {
|
||||
prepare(t, tmpDir, "fake-id3", []info{
|
||||
{path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{path: cgroup.CPUQuotaTotalFile, value: "100000"},
|
||||
{dir: "container1", path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{dir: "container1", path: cgroup.CPUQuotaTotalFile, value: "100000"},
|
||||
{dir: "container2", path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{dir: "container2", path: cgroup.CPUQuotaTotalFile, value: "-1"},
|
||||
})
|
||||
},
|
||||
post: func() map[string]string {
|
||||
return file.ReadBatchFromFile([]string{
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id3/cpu.cfs_burst_us"),
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id3/container1/cpu.cfs_burst_us"),
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id3/container2/cpu.cfs_burst_us"),
|
||||
})
|
||||
},
|
||||
wantErr: false,
|
||||
wantVal: map[string]string{
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id3/cpu.cfs_burst_us"): "50000",
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id3/container1/cpu.cfs_burst_us"): "50000",
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id3/container2/cpu.cfs_burst_us"): "0",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "quota burst < one container's quota, set min quota burst",
|
||||
event: framework.PodEvent{
|
||||
UID: "fake-id4",
|
||||
QoSLevel: 0,
|
||||
QoSClass: "",
|
||||
Pod: getPod("100000", "true"),
|
||||
},
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
prepare: func() {
|
||||
prepare(t, tmpDir, "fake-id4", []info{
|
||||
{path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{path: cgroup.CPUQuotaTotalFile, value: "300000"},
|
||||
{dir: "container1", path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{dir: "container1", path: cgroup.CPUQuotaTotalFile, value: "100000"},
|
||||
{dir: "container2", path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{dir: "container2", path: cgroup.CPUQuotaTotalFile, value: "200000"},
|
||||
})
|
||||
},
|
||||
post: func() map[string]string {
|
||||
return file.ReadBatchFromFile([]string{
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id4/cpu.cfs_burst_us"),
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id4/container1/cpu.cfs_burst_us"),
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id4/container2/cpu.cfs_burst_us"),
|
||||
})
|
||||
},
|
||||
wantErr: false,
|
||||
wantVal: map[string]string{
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id4/cpu.cfs_burst_us"): "200000",
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id4/container1/cpu.cfs_burst_us"): "100000",
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id4/container2/cpu.cfs_burst_us"): "100000",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "all containers contains quota!=-1, set quota burst successfully",
|
||||
event: framework.PodEvent{
|
||||
UID: "fake-id5",
|
||||
QoSLevel: 0,
|
||||
QoSClass: "",
|
||||
Pod: getPod("200000", "true"),
|
||||
},
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
prepare: func() {
|
||||
prepare(t, tmpDir, "fake-id5", []info{
|
||||
{path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{path: cgroup.CPUQuotaTotalFile, value: "300000"},
|
||||
{dir: "container1", path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{dir: "container1", path: cgroup.CPUQuotaTotalFile, value: "100000"},
|
||||
{dir: "container2", path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{dir: "container2", path: cgroup.CPUQuotaTotalFile, value: "200000"},
|
||||
})
|
||||
},
|
||||
post: func() map[string]string {
|
||||
return file.ReadBatchFromFile([]string{
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id5/cpu.cfs_burst_us"),
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id5/container1/cpu.cfs_burst_us"),
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id5/container2/cpu.cfs_burst_us"),
|
||||
})
|
||||
},
|
||||
wantErr: false,
|
||||
wantVal: map[string]string{
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id5/cpu.cfs_burst_us"): "300000",
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id5/container1/cpu.cfs_burst_us"): "100000",
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id5/container2/cpu.cfs_burst_us"): "200000",
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "all containers contains quota!=-1, set quota burst successfully",
|
||||
event: framework.PodEvent{
|
||||
UID: "fake-id6",
|
||||
QoSLevel: 0,
|
||||
QoSClass: "",
|
||||
Pod: getPod("200000", "false"),
|
||||
},
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
prepare: func() {
|
||||
prepare(t, tmpDir, "fake-id6", []info{
|
||||
{path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{path: cgroup.CPUQuotaTotalFile, value: "300000"},
|
||||
{dir: "container1", path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{dir: "container1", path: cgroup.CPUQuotaTotalFile, value: "100000"},
|
||||
{dir: "container2", path: cgroup.CPUQuotaBurstFile, value: "0"},
|
||||
{dir: "container2", path: cgroup.CPUQuotaTotalFile, value: "200000"},
|
||||
})
|
||||
},
|
||||
post: func() map[string]string {
|
||||
return file.ReadBatchFromFile([]string{
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id6/cpu.cfs_burst_us"),
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id6/container1/cpu.cfs_burst_us"),
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id6/container2/cpu.cfs_burst_us"),
|
||||
})
|
||||
},
|
||||
wantErr: false,
|
||||
wantVal: map[string]string{
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id6/cpu.cfs_burst_us"): "0",
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id6/container1/cpu.cfs_burst_us"): "0",
|
||||
path.Join(tmpDir, "cpu/kubepods/podfake-id6/container2/cpu.cfs_burst_us"): "0",
|
||||
},
|
||||
},
|
||||
}
|
||||
fakeClient := fake.NewSimpleClientset()
|
||||
informerFactory := informers.NewSharedInformerFactory(fakeClient, 0)
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
c := &CPUBurstHandle{
|
||||
cgroupMgr: tt.cgroupMgr,
|
||||
podInformer: informerFactory.Core().V1().Pods(),
|
||||
}
|
||||
if tt.prepare != nil {
|
||||
tt.prepare()
|
||||
}
|
||||
if err := c.Handle(tt.event); (err != nil) != tt.wantErr {
|
||||
t.Errorf("Handle() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
if tt.post != nil {
|
||||
assert.Equal(t, tt.wantVal, tt.post())
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func getPod(cpuQuotaBurst string, enableBurst string) *corev1.Pod {
|
||||
return &corev1.Pod{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Annotations: map[string]string{
|
||||
"volcano.sh/enable-quota-burst": enableBurst,
|
||||
"volcano.sh/quota-burst-time": cpuQuotaBurst,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
type info struct {
|
||||
dir string
|
||||
path string
|
||||
value string
|
||||
}
|
||||
|
||||
func prepare(t *testing.T, tmpDir, podUID string, infos []info) {
|
||||
for _, info := range infos {
|
||||
dir := path.Join(tmpDir, "cpu", "kubepods", "pod"+podUID, info.dir)
|
||||
err := os.MkdirAll(dir, 0644)
|
||||
assert.NoError(t, err)
|
||||
filePath := path.Join(dir, info.path)
|
||||
f, err := os.OpenFile(filePath, os.O_RDWR|os.O_CREATE, 0644)
|
||||
assert.NoError(t, err)
|
||||
err = f.Chmod(0600)
|
||||
assert.NoError(t, err)
|
||||
_, err = f.WriteString(info.value)
|
||||
assert.NoError(t, err)
|
||||
err = f.Close()
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,24 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package cpuburst
|
||||
|
||||
const (
|
||||
EnabledKey = "volcano.sh/enable-quota-burst"
|
||||
QuotaTimeKey = "volcano.sh/quota-burst-time"
|
||||
|
||||
fixedQuotaValue = -1
|
||||
)
|
||||
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package cpuqos
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers/base"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/agent/utils"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
func init() {
|
||||
handlers.RegisterEventHandleFunc(string(framework.PodEventName), NewCPUQoSHandle)
|
||||
}
|
||||
|
||||
type CPUQoSHandle struct {
|
||||
*base.BaseHandle
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
}
|
||||
|
||||
func NewCPUQoSHandle(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, cgroupMgr cgroup.CgroupManager) framework.Handle {
|
||||
return &CPUQoSHandle{
|
||||
BaseHandle: &base.BaseHandle{
|
||||
Name: string(features.CPUQoSFeature),
|
||||
Config: config,
|
||||
},
|
||||
cgroupMgr: cgroupMgr,
|
||||
}
|
||||
}
|
||||
|
||||
func (h *CPUQoSHandle) Handle(event interface{}) error {
|
||||
podEvent, ok := event.(framework.PodEvent)
|
||||
if !ok {
|
||||
return fmt.Errorf("illegal pod event")
|
||||
}
|
||||
|
||||
cgroupPath, err := h.cgroupMgr.GetPodCgroupPath(podEvent.QoSClass, cgroup.CgroupCpuSubsystem, podEvent.UID)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to get pod cgroup file(%s), error: %v", podEvent.UID, err)
|
||||
}
|
||||
qosLevelFile := path.Join(cgroupPath, cgroup.CPUQoSLevelFile)
|
||||
qosLevel := []byte(fmt.Sprintf("%d", podEvent.QoSLevel))
|
||||
|
||||
err = utils.UpdatePodCgroup(qosLevelFile, qosLevel)
|
||||
if err != nil {
|
||||
if errors.Is(err, os.ErrNotExist) {
|
||||
klog.InfoS("Cgroup file not existed", "cgroupFile", qosLevelFile)
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
klog.InfoS("Successfully set cpu qos level to cgroup file", "qosLevel", podEvent.QoSLevel, "cgroupFile", qosLevelFile)
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package cpuqos
|
||||
|
||||
import (
|
||||
"os"
|
||||
"path"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
)
|
||||
|
||||
func TestCPUQoSHandle_Handle(t *testing.T) {
|
||||
// make a fake pod cgroup path first.
|
||||
tmpDir := t.TempDir()
|
||||
dir := path.Join(tmpDir, "cpu", "kubepods", "podfake-id2")
|
||||
err := os.MkdirAll(dir, 0644)
|
||||
assert.NoError(t, err)
|
||||
filePath := path.Join(dir, "cpu.qos_level")
|
||||
_, err = os.OpenFile(filePath, os.O_RDWR|os.O_CREATE, 0644)
|
||||
assert.NoError(t, err)
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
event framework.PodEvent
|
||||
post func() int64
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
name: "cgroup path not exits, return no err",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
event: framework.PodEvent{
|
||||
UID: "fake-id1",
|
||||
QoSLevel: -1,
|
||||
QoSClass: "Guaranteed",
|
||||
},
|
||||
wantErr: false,
|
||||
},
|
||||
{
|
||||
name: "update cgroup path successfully, return no err",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
event: framework.PodEvent{
|
||||
UID: "fake-id2",
|
||||
QoSLevel: -1,
|
||||
QoSClass: "Guaranteed",
|
||||
},
|
||||
post: func() int64 {
|
||||
value, _ := os.ReadFile(filePath)
|
||||
i, _ := strconv.Atoi(string(value))
|
||||
return int64(i)
|
||||
},
|
||||
wantErr: false,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
h := &CPUQoSHandle{
|
||||
cgroupMgr: tt.cgroupMgr,
|
||||
}
|
||||
if err := h.Handle(tt.event); (err != nil) != tt.wantErr {
|
||||
t.Errorf("Handle() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
if tt.post != nil {
|
||||
assert.Equal(t, tt.event.QoSLevel, tt.post())
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package eviction
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"reflect"
|
||||
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/queue"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/agent/utils/eviction"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
func init() {
|
||||
handlers.RegisterEventHandleFunc(string(framework.NodeMonitorEventName), NewManager)
|
||||
}
|
||||
|
||||
type manager struct {
|
||||
cfg *config.Configuration
|
||||
eviction.Eviction
|
||||
policy.Interface
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
getPodsFunc utilpod.ActivePods
|
||||
}
|
||||
|
||||
func NewManager(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, cgroupMgr cgroup.CgroupManager) framework.Handle {
|
||||
evictor := eviction.NewEviction(config.GenericConfiguration.KubeClient, config.GenericConfiguration.KubeNodeName)
|
||||
m := &manager{
|
||||
cfg: config,
|
||||
Eviction: evictor,
|
||||
Interface: policy.GetPolicyFunc(config.GenericConfiguration.OverSubscriptionPolicy)(config, mgr, evictor, queue.NewSqQueue(), ""),
|
||||
getNodeFunc: config.GetNode,
|
||||
getPodsFunc: config.GetActivePods,
|
||||
}
|
||||
return m
|
||||
}
|
||||
|
||||
func (m *manager) Handle(event interface{}) error {
|
||||
nodeMonitorEvent, ok := event.(framework.NodeMonitorEvent)
|
||||
if !ok {
|
||||
klog.ErrorS(nil, "Invalid node monitor event", "type", reflect.TypeOf(event))
|
||||
return nil
|
||||
}
|
||||
|
||||
klog.InfoS("Received node pressure event", "resource", nodeMonitorEvent.Resource, "time", nodeMonitorEvent.TimeStamp)
|
||||
node, err := m.getNodeFunc()
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get node")
|
||||
return err
|
||||
}
|
||||
nodeCopy := node.DeepCopy()
|
||||
|
||||
for _, res := range apis.OverSubscriptionResourceTypes {
|
||||
if res != nodeMonitorEvent.Resource {
|
||||
continue
|
||||
}
|
||||
preemptablePods, _, err := utilnode.GetLatestPodsAndResList(nodeCopy, m.getPodsFunc, res)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get pods and resource list")
|
||||
return err
|
||||
}
|
||||
|
||||
for _, pod := range preemptablePods {
|
||||
if err = m.DisableSchedule(); err != nil {
|
||||
klog.ErrorS(err, "Failed to add eviction annotation")
|
||||
}
|
||||
klog.InfoS("Successfully disable schedule")
|
||||
|
||||
klog.InfoS("Try to evict pod", "pod", klog.KObj(pod))
|
||||
if m.Evict(context.TODO(), pod, m.cfg.GenericConfiguration.Recorder, 0, fmt.Sprintf("Evict offline pod due to %s resource pressure", res)) {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *manager) RefreshCfg(cfg *api.ColocationConfig) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *manager) IsActive() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (m *manager) HandleName() string {
|
||||
return string(features.EvictionFeature)
|
||||
}
|
||||
|
|
@ -0,0 +1,135 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package eviction
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/agiledragon/gomonkey/v2"
|
||||
"github.com/stretchr/testify/assert"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
fakeclientset "k8s.io/client-go/kubernetes/fake"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy/extend"
|
||||
"volcano.sh/volcano/pkg/agent/utils/eviction"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
utiltesting "volcano.sh/volcano/pkg/agent/utils/testing"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
)
|
||||
|
||||
func makeNode() (*v1.Node, error) {
|
||||
return &v1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "test-node",
|
||||
Labels: make(map[string]string),
|
||||
Annotations: make(map[string]string),
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
func Test_manager_Handle(t *testing.T) {
|
||||
fakeTime, err := time.Parse("2006-01-02", "2023-04-01")
|
||||
assert.NoError(t, err)
|
||||
patch := gomonkey.NewPatches()
|
||||
defer patch.Reset()
|
||||
patch.ApplyFunc(time.Now, func() time.Time {
|
||||
return fakeTime
|
||||
})
|
||||
|
||||
pp := utiltesting.NewPodProvider(
|
||||
utiltesting.MakePod("offline-pod-1", 30, 30, "BE"),
|
||||
utiltesting.MakePod("offline-pod-2", 40, 30, "BE"),
|
||||
utiltesting.MakePod("online-pod", 10, 10, ""),
|
||||
)
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
event interface{}
|
||||
Eviction eviction.Eviction
|
||||
policy func(cfg *config.Configuration, pods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
getPodsFunc utilpod.ActivePods
|
||||
wantErr assert.ErrorAssertionFunc
|
||||
expectedNode func() *v1.Node
|
||||
}{
|
||||
{
|
||||
name: "evict high request pod with extend resource",
|
||||
event: framework.NodeMonitorEvent{
|
||||
TimeStamp: time.Now(),
|
||||
Resource: v1.ResourceCPU,
|
||||
},
|
||||
Eviction: pp,
|
||||
policy: func(cfg *config.Configuration, pods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface {
|
||||
return extend.NewExtendResource(cfg, nil, evictor, nil, "")
|
||||
},
|
||||
getPodsFunc: pp.GetPodsFunc,
|
||||
getNodeFunc: makeNode,
|
||||
wantErr: assert.NoError,
|
||||
expectedNode: func() *v1.Node {
|
||||
node := &v1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "test-node",
|
||||
},
|
||||
Spec: v1.NodeSpec{Taints: []v1.Taint{{Key: apis.PodEvictingKey, Effect: v1.TaintEffectNoSchedule}}},
|
||||
}
|
||||
return node
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
fakeNode, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
fakeClient := fakeclientset.NewSimpleClientset(fakeNode)
|
||||
cfg := &config.Configuration{GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
KubeClient: fakeClient,
|
||||
KubeNodeName: "test-node",
|
||||
NodeHasSynced: func() bool {
|
||||
return false
|
||||
},
|
||||
}}
|
||||
m := &manager{
|
||||
cfg: cfg,
|
||||
Interface: tt.policy(cfg, tt.getPodsFunc, nil),
|
||||
Eviction: tt.Eviction,
|
||||
getNodeFunc: tt.getNodeFunc,
|
||||
getPodsFunc: tt.getPodsFunc,
|
||||
}
|
||||
tt.wantErr(t, m.Handle(tt.event), fmt.Sprintf("Handle(%v)", tt.event))
|
||||
evictedPods := pp.GetEvictedPods()
|
||||
// Verify that pod should be evicted.
|
||||
if len(evictedPods) == 0 || evictedPods[0].Name != "offline-pod-2" {
|
||||
t.Errorf("Manager should have evicted offline-pod-2 but not")
|
||||
}
|
||||
|
||||
// Node should be schedule disabled.
|
||||
node, err := fakeClient.CoreV1().Nodes().Get(context.TODO(), "test-node", metav1.GetOptions{})
|
||||
if err != nil {
|
||||
t.Errorf("Failed to get node, err: %v", err)
|
||||
}
|
||||
assert.Equalf(t, tt.expectedNode(), node, "Node should have eviction annotation or taint")
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,81 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package memoryqos
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis/extension"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers/base"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/agent/utils"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
func init() {
|
||||
handlers.RegisterEventHandleFunc(string(framework.PodEventName), NewMemoryQoSHandle)
|
||||
}
|
||||
|
||||
type MemoryQoSHandle struct {
|
||||
*base.BaseHandle
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
}
|
||||
|
||||
func NewMemoryQoSHandle(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, cgroupMgr cgroup.CgroupManager) framework.Handle {
|
||||
return &MemoryQoSHandle{
|
||||
BaseHandle: &base.BaseHandle{
|
||||
Name: string(features.MemoryQoSFeature),
|
||||
Config: config,
|
||||
},
|
||||
cgroupMgr: cgroupMgr,
|
||||
}
|
||||
}
|
||||
|
||||
func (h *MemoryQoSHandle) Handle(event interface{}) error {
|
||||
podEvent, ok := event.(framework.PodEvent)
|
||||
if !ok {
|
||||
return fmt.Errorf("illegal pod event")
|
||||
}
|
||||
|
||||
cgroupPath, err := h.cgroupMgr.GetPodCgroupPath(podEvent.QoSClass, cgroup.CgroupMemorySubsystem, podEvent.UID)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to get pod cgroup file(%s), error: %v", podEvent.UID, err)
|
||||
}
|
||||
qosLevelFile := path.Join(cgroupPath, cgroup.MemoryQoSLevelFile)
|
||||
qosLevel := []byte(fmt.Sprintf("%d", extension.NormalizeQosLevel(podEvent.QoSLevel)))
|
||||
|
||||
err = utils.UpdatePodCgroup(qosLevelFile, qosLevel)
|
||||
if err != nil {
|
||||
if errors.Is(err, os.ErrNotExist) {
|
||||
klog.InfoS("Cgroup file not existed", "cgroupFile", qosLevelFile)
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
klog.InfoS("Successfully set memory qos level to cgroup file", "qosLevel", qosLevel, "cgroupFile", qosLevelFile)
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,131 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package memoryqos
|
||||
|
||||
import (
|
||||
"os"
|
||||
"path"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
)
|
||||
|
||||
func TestMemroyQoSHandle_Handle(t *testing.T) {
|
||||
dir, err := os.MkdirTemp("/tmp", "MkdirTempCgroup")
|
||||
defer func() {
|
||||
err = os.RemoveAll(dir)
|
||||
if err != nil {
|
||||
t.Errorf("remove dir(%s) failed: %v", dir, err)
|
||||
}
|
||||
assert.Equal(t, err == nil, true)
|
||||
}()
|
||||
assert.Equal(t, err == nil, true)
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
cgroupSubpath string
|
||||
event framework.PodEvent
|
||||
expectedErr bool
|
||||
expectedQoSLevel string
|
||||
}{
|
||||
{
|
||||
name: "Burstable pod event && CgroupDriver=Cgroupfs",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", path.Join(dir, "cgroup"), ""),
|
||||
cgroupSubpath: "cgroup/memory/kubepods/burstable",
|
||||
event: framework.PodEvent{
|
||||
UID: "00000000-1111-2222-3333-000000000001",
|
||||
QoSLevel: -1,
|
||||
QoSClass: "Burstable",
|
||||
},
|
||||
expectedErr: false,
|
||||
expectedQoSLevel: "-1",
|
||||
},
|
||||
|
||||
{
|
||||
name: "Guaranteed pod event && CgroupDriver=Cgroupfs",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", path.Join(dir, "cgroup"), ""),
|
||||
cgroupSubpath: "cgroup/memory/kubepods",
|
||||
event: framework.PodEvent{
|
||||
UID: "00000000-1111-2222-3333-000000000002",
|
||||
QoSLevel: -1,
|
||||
QoSClass: "Guaranteed",
|
||||
},
|
||||
expectedErr: false,
|
||||
expectedQoSLevel: "-1",
|
||||
},
|
||||
|
||||
{
|
||||
name: "BestEffort pod event && CgroupDriver=Cgroupfs",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", path.Join(dir, "cgroup"), ""),
|
||||
cgroupSubpath: "cgroup/memory/kubepods/besteffort",
|
||||
event: framework.PodEvent{
|
||||
UID: "00000000-1111-2222-3333-000000000003",
|
||||
QoSLevel: -1,
|
||||
QoSClass: "BestEffort",
|
||||
},
|
||||
expectedErr: false,
|
||||
expectedQoSLevel: "-1",
|
||||
},
|
||||
{
|
||||
name: "BestEffort pod event && CgroupDriver=Cgroupfs qos level=-2",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", path.Join(dir, "cgroup"), ""),
|
||||
cgroupSubpath: "cgroup/memory/kubepods/besteffort",
|
||||
event: framework.PodEvent{
|
||||
UID: "00000000-1111-2222-3333-000000000003",
|
||||
QoSLevel: -2,
|
||||
QoSClass: "BestEffort",
|
||||
},
|
||||
expectedErr: false,
|
||||
expectedQoSLevel: "-1",
|
||||
},
|
||||
{
|
||||
name: "BestEffort pod event && CgroupDriver=Cgroupfs qos level=2",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", path.Join(dir, "cgroup"), ""),
|
||||
cgroupSubpath: "cgroup/memory/kubepods/besteffort",
|
||||
event: framework.PodEvent{
|
||||
UID: "00000000-1111-2222-3333-000000000003",
|
||||
QoSLevel: 2,
|
||||
QoSClass: "BestEffort",
|
||||
},
|
||||
expectedErr: false,
|
||||
expectedQoSLevel: "0",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
fakeCgroupPath := path.Join(dir, tc.cgroupSubpath, "pod"+string(tc.event.UID))
|
||||
err = os.MkdirAll(fakeCgroupPath, 0750)
|
||||
assert.Equal(t, err == nil, true, tc.name)
|
||||
|
||||
tmpFile := path.Join(fakeCgroupPath, "memory.qos_level")
|
||||
if err = os.WriteFile(tmpFile, []byte("0"), 0660); err != nil {
|
||||
assert.Equal(t, nil, err, tc.name)
|
||||
}
|
||||
|
||||
h := NewMemoryQoSHandle(nil, nil, tc.cgroupMgr)
|
||||
handleErr := h.Handle(tc.event)
|
||||
assert.Equal(t, tc.expectedErr, handleErr != nil, tc.name)
|
||||
|
||||
actualLevel, readErr := os.ReadFile(tmpFile)
|
||||
assert.Equal(t, nil, readErr, tc.name)
|
||||
assert.Equal(t, tc.expectedQoSLevel, string(actualLevel), tc.name)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,135 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package networkqos
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
"strconv"
|
||||
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/errors"
|
||||
listersv1 "k8s.io/client-go/listers/core/v1"
|
||||
"k8s.io/client-go/tools/record"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis/extension"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers/base"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/agent/utils"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
"volcano.sh/volcano/pkg/networkqos"
|
||||
)
|
||||
|
||||
func init() {
|
||||
handlers.RegisterEventHandleFunc(string(framework.PodEventName), NewNetworkQoSHandle)
|
||||
}
|
||||
|
||||
type NetworkQoSHandle struct {
|
||||
*base.BaseHandle
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
networkqosMgr networkqos.NetworkQoSManager
|
||||
poLister listersv1.PodLister
|
||||
recorder record.EventRecorder
|
||||
}
|
||||
|
||||
func NewNetworkQoSHandle(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, cgroupMgr cgroup.CgroupManager) framework.Handle {
|
||||
return &NetworkQoSHandle{
|
||||
BaseHandle: &base.BaseHandle{
|
||||
Name: string(features.NetworkQoSFeature),
|
||||
Config: config,
|
||||
},
|
||||
cgroupMgr: cgroupMgr,
|
||||
networkqosMgr: networkqos.GetNetworkQoSManager(config),
|
||||
poLister: config.InformerFactory.K8SInformerFactory.Core().V1().Pods().Lister(),
|
||||
recorder: config.GenericConfiguration.Recorder,
|
||||
}
|
||||
}
|
||||
|
||||
func (h *NetworkQoSHandle) Handle(event interface{}) error {
|
||||
podEvent, ok := event.(framework.PodEvent)
|
||||
if !ok {
|
||||
return fmt.Errorf("illegal pod event: %v", event)
|
||||
}
|
||||
|
||||
pod, err := h.poLister.Pods(podEvent.Pod.Namespace).Get(podEvent.Pod.Name)
|
||||
if err != nil {
|
||||
if !errors.IsNotFound(err) {
|
||||
klog.V(4).InfoS("pod does not existed, skipped handling network qos", "namespace", podEvent.Pod.Namespace, "name", podEvent.Pod.Name)
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
_, ingressExisted := pod.Annotations["kubernetes.io/ingress-bandwidth"]
|
||||
_, egressExisted := pod.Annotations["kubernetes.io/egress-bandwidth"]
|
||||
if ingressExisted || egressExisted {
|
||||
h.recorder.Event(pod, corev1.EventTypeWarning, "NetworkQoSSkipped",
|
||||
fmt.Sprintf("Colocation Network QoS is not set, because it already has an Ingress-Bandwidth/Egress-Bandwidth"+
|
||||
" network rate limit(with annotation key kubernetes.io/ingress-bandwidth or kubernetes.io/egress-bandwidth )"))
|
||||
return nil
|
||||
}
|
||||
|
||||
cgroupPath, err := h.cgroupMgr.GetPodCgroupPath(podEvent.QoSClass, cgroup.CgroupNetCLSSubsystem, podEvent.UID)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to get pod cgroup file(%s), error: %v", podEvent.UID, err)
|
||||
}
|
||||
|
||||
qosLevelFile := path.Join(cgroupPath, cgroup.NetCLSFileName)
|
||||
uintQoSLevel := uint32(extension.NormalizeQosLevel(podEvent.QoSLevel))
|
||||
qosLevel := []byte(strconv.FormatUint(uint64(uintQoSLevel), 10))
|
||||
|
||||
err = utils.UpdatePodCgroup(qosLevelFile, qosLevel)
|
||||
if os.IsNotExist(err) {
|
||||
klog.InfoS("Cgroup file not existed", "cgroupFile", qosLevelFile)
|
||||
return nil
|
||||
}
|
||||
klog.InfoS("Successfully set network qos level to cgroup file", "qosLevel", string(qosLevel), "cgroupFile", qosLevelFile)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *NetworkQoSHandle) RefreshCfg(cfg *api.ColocationConfig) error {
|
||||
if err := h.BaseHandle.RefreshCfg(cfg); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
h.Lock.Lock()
|
||||
defer h.Lock.Unlock()
|
||||
if h.Active {
|
||||
err := h.networkqosMgr.EnableNetworkQoS(cfg.NetworkQosConfig)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to enable network qos")
|
||||
return err
|
||||
}
|
||||
klog.V(5).InfoS("Successfully enable/update network QoS")
|
||||
return nil
|
||||
}
|
||||
|
||||
err := h.networkqosMgr.DisableNetworkQoS()
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to disable network qos")
|
||||
return err
|
||||
}
|
||||
klog.V(5).InfoS("Successfully disable network QoS")
|
||||
return nil
|
||||
}
|
||||
|
|
@ -0,0 +1,154 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package networkqos
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/client-go/informers"
|
||||
"k8s.io/client-go/kubernetes/fake"
|
||||
"k8s.io/client-go/tools/cache"
|
||||
"k8s.io/client-go/tools/record"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
)
|
||||
|
||||
func TestNeworkQoSHandle_Handle(t *testing.T) {
|
||||
dir, err := os.MkdirTemp("/tmp", "MkdirTempCgroup")
|
||||
defer func() {
|
||||
err = os.RemoveAll(dir)
|
||||
if err != nil {
|
||||
t.Errorf("remove dir(%s) failed: %v", dir, err)
|
||||
}
|
||||
assert.Equal(t, err == nil, true)
|
||||
}()
|
||||
assert.Equal(t, err == nil, true)
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
cgroupSubpath string
|
||||
recorder record.EventRecorder
|
||||
event framework.PodEvent
|
||||
expectedErr bool
|
||||
expectedQoSLevel string
|
||||
}{
|
||||
{
|
||||
name: "Burstable pod event && CgroupDriver=Cgroupfs",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", path.Join(dir, "cgroup"), ""),
|
||||
cgroupSubpath: "cgroup/net_cls/kubepods/burstable",
|
||||
event: framework.PodEvent{
|
||||
UID: "00000000-1111-2222-3333-000000000001",
|
||||
QoSLevel: -1,
|
||||
QoSClass: "Burstable",
|
||||
Pod: &corev1.Pod{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "test1",
|
||||
Namespace: "default",
|
||||
},
|
||||
},
|
||||
},
|
||||
expectedErr: false,
|
||||
expectedQoSLevel: "4294967295",
|
||||
},
|
||||
|
||||
{
|
||||
name: "Guaranteed pod event && CgroupDriver=Cgroupfs",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", path.Join(dir, "cgroup"), ""),
|
||||
cgroupSubpath: "cgroup/net_cls/kubepods",
|
||||
event: framework.PodEvent{
|
||||
UID: "00000000-1111-2222-3333-000000000002",
|
||||
QoSLevel: -1,
|
||||
QoSClass: "Guaranteed",
|
||||
Pod: &corev1.Pod{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "test2",
|
||||
Namespace: "default",
|
||||
},
|
||||
},
|
||||
},
|
||||
expectedErr: false,
|
||||
expectedQoSLevel: "4294967295",
|
||||
},
|
||||
|
||||
{
|
||||
name: "BestEffort pod event && CgroupDriver=Cgroupfs",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", path.Join(dir, "cgroup"), ""),
|
||||
cgroupSubpath: "cgroup/net_cls/kubepods/besteffort",
|
||||
event: framework.PodEvent{
|
||||
UID: "00000000-1111-2222-3333-000000000003",
|
||||
QoSLevel: -1,
|
||||
QoSClass: "BestEffort",
|
||||
Pod: &corev1.Pod{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "test3",
|
||||
Namespace: "default",
|
||||
},
|
||||
},
|
||||
},
|
||||
expectedErr: false,
|
||||
expectedQoSLevel: "4294967295",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
fakeCgroupPath := path.Join(dir, tc.cgroupSubpath, "pod"+string(tc.event.UID))
|
||||
err = os.MkdirAll(fakeCgroupPath, 0750)
|
||||
assert.Equal(t, err == nil, true, tc.name)
|
||||
|
||||
tmpFile := path.Join(fakeCgroupPath, "net_cls.classid")
|
||||
if err = os.WriteFile(tmpFile, []byte("0"), 0660); err != nil {
|
||||
assert.Equal(t, nil, err, tc.name)
|
||||
}
|
||||
|
||||
fakeClient := fake.NewSimpleClientset(tc.event.Pod)
|
||||
informerFactory := informers.NewSharedInformerFactory(fakeClient, 0)
|
||||
informerFactory.Core().V1().Pods().Informer()
|
||||
informerFactory.Start(context.TODO().Done())
|
||||
if !cache.WaitForNamedCacheSync("", context.TODO().Done(), informerFactory.Core().V1().Pods().Informer().HasSynced) {
|
||||
assert.Equal(t, nil, err, tc.name)
|
||||
}
|
||||
|
||||
cfg := &config.Configuration{
|
||||
InformerFactory: &config.InformerFactory{
|
||||
K8SInformerFactory: informerFactory,
|
||||
},
|
||||
GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
KubeClient: fakeClient,
|
||||
Recorder: record.NewFakeRecorder(100),
|
||||
},
|
||||
}
|
||||
|
||||
h := NewNetworkQoSHandle(cfg, nil, tc.cgroupMgr)
|
||||
handleErr := h.Handle(tc.event)
|
||||
fmt.Println(handleErr)
|
||||
assert.Equal(t, tc.expectedErr, handleErr != nil, tc.name)
|
||||
|
||||
actualLevel, readErr := os.ReadFile(tmpFile)
|
||||
assert.Equal(t, nil, readErr, tc.name)
|
||||
assert.Equal(t, tc.expectedQoSLevel, string(actualLevel), tc.name)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,151 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package oversubscription
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers/base"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/agent/metrics"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/agent/utils/eviction"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
const (
|
||||
reSyncPeriod = 6 // report overSubscription resources every 10s, so 6 represents 1 minutes.
|
||||
)
|
||||
|
||||
func init() {
|
||||
handlers.RegisterEventHandleFunc(string(framework.NodeResourcesEventName), NewReporter)
|
||||
}
|
||||
|
||||
type reporter struct {
|
||||
*base.BaseHandle
|
||||
policy.Interface
|
||||
enabled bool
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
getPodsFunc utilpod.ActivePods
|
||||
killPodFunc utilpod.KillPod
|
||||
reportTimes int64
|
||||
}
|
||||
|
||||
func NewReporter(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, cgroupMgr cgroup.CgroupManager) framework.Handle {
|
||||
evictor := eviction.NewEviction(config.GenericConfiguration.KubeClient, config.GenericConfiguration.KubeNodeName)
|
||||
return &reporter{
|
||||
BaseHandle: &base.BaseHandle{
|
||||
Name: string(features.OverSubscriptionFeature),
|
||||
Config: config,
|
||||
},
|
||||
Interface: policy.GetPolicyFunc(config.GenericConfiguration.OverSubscriptionPolicy)(config, nil, evictor, nil, ""),
|
||||
enabled: true,
|
||||
getPodsFunc: config.GetActivePods,
|
||||
getNodeFunc: config.GetNode,
|
||||
}
|
||||
}
|
||||
|
||||
func (r *reporter) Handle(event interface{}) error {
|
||||
nodeResourceEvent, ok := event.(framework.NodeResourceEvent)
|
||||
if !ok {
|
||||
klog.ErrorS(nil, "Invalid node resource event", "type", reflect.TypeOf(event))
|
||||
return nil
|
||||
}
|
||||
|
||||
overSubRes := apis.Resource{
|
||||
corev1.ResourceCPU: nodeResourceEvent.MillCPU,
|
||||
corev1.ResourceMemory: nodeResourceEvent.MemoryBytes,
|
||||
}
|
||||
|
||||
node, err := r.getNodeFunc()
|
||||
if err != nil {
|
||||
klog.ErrorS(nil, "overSubscription: failed to get node")
|
||||
return err
|
||||
}
|
||||
nodeCopy := node.DeepCopy()
|
||||
|
||||
if !utilnode.IsNodeSupportOverSubscription(nodeCopy) {
|
||||
return nil
|
||||
}
|
||||
|
||||
r.reportTimes++
|
||||
if !r.shouldPatchOverSubscription(nodeCopy, overSubRes) {
|
||||
return nil
|
||||
}
|
||||
|
||||
if err = r.UpdateOverSubscription(overSubRes); err != nil {
|
||||
klog.ErrorS(err, "OverSubscription: failed to update overSubscription resource")
|
||||
return nil
|
||||
}
|
||||
metrics.UpdateOverSubscriptionResourceQuantity(r.Config.GenericConfiguration.KubeNodeName, overSubRes)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *reporter) RefreshCfg(cfg *api.ColocationConfig) error {
|
||||
if err := r.BaseHandle.RefreshCfg(cfg); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
r.Lock.Lock()
|
||||
defer r.Lock.Unlock()
|
||||
|
||||
emptyRes := apis.Resource{
|
||||
corev1.ResourceCPU: 0,
|
||||
corev1.ResourceMemory: 0,
|
||||
}
|
||||
// OverSubscriptionConfig is disabled.
|
||||
if !*cfg.OverSubscriptionConfig.Enable {
|
||||
r.enabled = false
|
||||
metrics.UpdateOverSubscriptionResourceQuantity(r.Config.GenericConfiguration.KubeNodeName, emptyRes)
|
||||
return r.Cleanup()
|
||||
}
|
||||
|
||||
// OverSubscriptionConfig is enabled, we should check node label because overSubscription can be turned off by reset node label, too.
|
||||
// node colocation and subscription are turned off by nodePool setting, do clean up.
|
||||
if r.enabled && !*cfg.NodeLabelConfig.NodeOverSubscriptionEnable {
|
||||
metrics.UpdateOverSubscriptionResourceQuantity(r.Config.GenericConfiguration.KubeNodeName, emptyRes)
|
||||
return r.Cleanup()
|
||||
} else {
|
||||
// overSubscription is turned on by OverSubscriptionConfig, set node label.
|
||||
if err := utilnode.SetOverSubscriptionLabel(r.Config); err != nil {
|
||||
klog.ErrorS(err, "Failed to set overSubscription label")
|
||||
return err
|
||||
}
|
||||
r.enabled = true
|
||||
klog.InfoS("Successfully set overSubscription label=true")
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (r *reporter) shouldPatchOverSubscription(node *corev1.Node, usage apis.Resource) bool {
|
||||
// re-sync overSubscription resources every 1 minute.
|
||||
if r.reportTimes%reSyncPeriod == 0 {
|
||||
return true
|
||||
}
|
||||
return r.ShouldUpdateOverSubscription(node, usage)
|
||||
}
|
||||
|
|
@ -0,0 +1,254 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package oversubscription
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/resource"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/util/sets"
|
||||
fakeclientset "k8s.io/client-go/kubernetes/fake"
|
||||
utilpointer "k8s.io/utils/pointer"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers/base"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy/extend"
|
||||
"volcano.sh/volcano/pkg/agent/utils/eviction"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
utiltesting "volcano.sh/volcano/pkg/agent/utils/testing"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
)
|
||||
|
||||
func makeNode() (*v1.Node, error) {
|
||||
return &v1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "test-node",
|
||||
Annotations: map[string]string{
|
||||
apis.OverSubscriptionTypesKey: "cpu",
|
||||
},
|
||||
Labels: map[string]string{
|
||||
apis.OverSubscriptionNodeLabelKey: "true",
|
||||
},
|
||||
},
|
||||
Status: v1.NodeStatus{Allocatable: map[v1.ResourceName]resource.Quantity{
|
||||
v1.ResourceCPU: *resource.NewQuantity(10000, resource.DecimalSI),
|
||||
v1.ResourceMemory: *resource.NewQuantity(10000, resource.BinarySI),
|
||||
}},
|
||||
}, nil
|
||||
}
|
||||
|
||||
func Test_reporter_RefreshCfg(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
getPodsFunc utilpod.ActivePods
|
||||
podProvider *utiltesting.PodProvider
|
||||
policy func(cfg *config.Configuration, pods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface
|
||||
cfg *api.ColocationConfig
|
||||
enable bool
|
||||
expectNode func() *v1.Node
|
||||
expectTypes sets.String
|
||||
expectEvictedPods []*v1.Pod
|
||||
}{
|
||||
{
|
||||
name: "enable overSubscription",
|
||||
getNodeFunc: func() (*v1.Node, error) {
|
||||
node, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
node.Labels[apis.OverSubscriptionNodeLabelKey] = "false"
|
||||
return node, nil
|
||||
},
|
||||
podProvider: utiltesting.NewPodProvider(),
|
||||
policy: func(cfg *config.Configuration, activePods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface {
|
||||
return extend.NewExtendResource(cfg, nil, evictor, nil, "")
|
||||
},
|
||||
cfg: &api.ColocationConfig{
|
||||
NodeLabelConfig: &api.NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(false),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(true),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String("cpu,memory"),
|
||||
}},
|
||||
enable: true,
|
||||
expectTypes: sets.NewString("cpu", "memory"),
|
||||
expectNode: func() *v1.Node {
|
||||
node, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
return node
|
||||
},
|
||||
expectEvictedPods: nil,
|
||||
},
|
||||
{
|
||||
name: "enable overSubscription after disable config",
|
||||
getNodeFunc: func() (*v1.Node, error) {
|
||||
node, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
node.Labels[apis.OverSubscriptionNodeLabelKey] = "false"
|
||||
return node, nil
|
||||
},
|
||||
podProvider: utiltesting.NewPodProvider(),
|
||||
policy: func(cfg *config.Configuration, activePods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface {
|
||||
return extend.NewExtendResource(cfg, nil, evictor, nil, "")
|
||||
},
|
||||
cfg: &api.ColocationConfig{
|
||||
NodeLabelConfig: &api.NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(false),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(false),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
OverSubscriptionTypes: utilpointer.String("cpu,memory"),
|
||||
}},
|
||||
expectTypes: sets.NewString("cpu", "memory"),
|
||||
expectNode: func() *v1.Node {
|
||||
node, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
return node
|
||||
},
|
||||
expectEvictedPods: nil,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
fakeNode, _ := tt.getNodeFunc()
|
||||
fakeClient := fakeclientset.NewSimpleClientset(fakeNode)
|
||||
cfg := &config.Configuration{GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
KubeClient: fakeClient,
|
||||
KubeNodeName: "test-node",
|
||||
NodeHasSynced: func() bool {
|
||||
return false
|
||||
},
|
||||
SupportedFeatures: []string{string(features.OverSubscriptionFeature)},
|
||||
}}
|
||||
r := &reporter{
|
||||
BaseHandle: &base.BaseHandle{
|
||||
Name: string(features.OverSubscriptionFeature),
|
||||
Config: cfg,
|
||||
},
|
||||
Interface: tt.policy(cfg, tt.podProvider.GetPodsFunc, tt.podProvider),
|
||||
enabled: tt.enable,
|
||||
getNodeFunc: tt.getNodeFunc,
|
||||
getPodsFunc: tt.podProvider.GetPodsFunc,
|
||||
}
|
||||
assert.NoError(t, r.RefreshCfg(tt.cfg))
|
||||
node, err := fakeClient.CoreV1().Nodes().Get(context.TODO(), "test-node", metav1.GetOptions{})
|
||||
if err != nil {
|
||||
t.Errorf("Failed to get node, err: %v", err)
|
||||
}
|
||||
assert.Equal(t, tt.expectNode(), node)
|
||||
assert.Equal(t, tt.expectEvictedPods, tt.podProvider.GetEvictedPods())
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func Test_reporter_Handle(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
policy func(cfg *config.Configuration, pods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
getPodsFunc utilpod.ActivePods
|
||||
killPodFunc utilpod.KillPod
|
||||
event interface{}
|
||||
wantErr assert.ErrorAssertionFunc
|
||||
expectedNode func() *v1.Node
|
||||
}{
|
||||
{
|
||||
name: "not over subscription node, skip patch",
|
||||
policy: func(cfg *config.Configuration, pods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface {
|
||||
return extend.NewExtendResource(cfg, nil, evictor, nil, "")
|
||||
},
|
||||
getNodeFunc: func() (*v1.Node, error) {
|
||||
node, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
node.Labels[apis.OverSubscriptionNodeLabelKey] = "false"
|
||||
return node, nil
|
||||
},
|
||||
event: framework.NodeResourceEvent{
|
||||
MillCPU: 1000,
|
||||
MemoryBytes: 2000,
|
||||
},
|
||||
wantErr: assert.NoError,
|
||||
expectedNode: func() *v1.Node {
|
||||
node, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
node.Labels[apis.OverSubscriptionNodeLabelKey] = "false"
|
||||
return node
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "patch over subscription node to node status",
|
||||
policy: func(cfg *config.Configuration, pods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface {
|
||||
return extend.NewExtendResource(cfg, nil, evictor, nil, "")
|
||||
},
|
||||
getNodeFunc: makeNode,
|
||||
event: framework.NodeResourceEvent{
|
||||
MillCPU: 1000,
|
||||
MemoryBytes: 2000,
|
||||
},
|
||||
wantErr: assert.NoError,
|
||||
expectedNode: func() *v1.Node {
|
||||
node, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
node.Status.Capacity = map[v1.ResourceName]resource.Quantity{}
|
||||
node.Status.Capacity[apis.ExtendResourceCPU] = *resource.NewQuantity(1000, resource.DecimalSI)
|
||||
node.Status.Capacity[apis.ExtendResourceMemory] = *resource.NewQuantity(2000, resource.BinarySI)
|
||||
node.Status.Allocatable[apis.ExtendResourceCPU] = *resource.NewQuantity(1000, resource.DecimalSI)
|
||||
node.Status.Allocatable[apis.ExtendResourceMemory] = *resource.NewQuantity(2000, resource.BinarySI)
|
||||
return node
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
makeNode, _ := tt.getNodeFunc()
|
||||
fakeClient := fakeclientset.NewSimpleClientset(makeNode)
|
||||
cfg := &config.Configuration{GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
KubeClient: fakeClient,
|
||||
KubeNodeName: "test-node",
|
||||
NodeHasSynced: func() bool {
|
||||
return false
|
||||
},
|
||||
}}
|
||||
r := &reporter{
|
||||
BaseHandle: &base.BaseHandle{
|
||||
Config: cfg},
|
||||
Interface: tt.policy(cfg, tt.getPodsFunc, nil),
|
||||
getNodeFunc: tt.getNodeFunc,
|
||||
getPodsFunc: tt.getPodsFunc,
|
||||
killPodFunc: tt.killPodFunc,
|
||||
}
|
||||
tt.wantErr(t, r.Handle(tt.event), fmt.Sprintf("Handle(%v)", tt.event))
|
||||
node, err := fakeClient.CoreV1().Nodes().Get(context.TODO(), "test-node", metav1.GetOptions{})
|
||||
if err != nil {
|
||||
t.Errorf("Failed to get node, err: %v", err)
|
||||
}
|
||||
assert.Equal(t, tt.expectedNode(), node)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package handlers
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
var handlerFuncs = map[string][]NewEventHandleFunc{}
|
||||
var mutex sync.Mutex
|
||||
|
||||
type NewEventHandleFunc = func(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, cgroupMgr cgroup.CgroupManager) framework.Handle
|
||||
|
||||
func RegisterEventHandleFunc(eventName string, f NewEventHandleFunc) {
|
||||
mutex.Lock()
|
||||
defer mutex.Unlock()
|
||||
|
||||
handlerFuncs[eventName] = append(handlerFuncs[eventName], f)
|
||||
}
|
||||
|
||||
func GetEventHandlerFuncs() map[string][]NewEventHandleFunc {
|
||||
return handlerFuncs
|
||||
}
|
||||
|
|
@ -0,0 +1,100 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package resources
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
"strconv"
|
||||
|
||||
utilerrors "k8s.io/apimachinery/pkg/util/errors"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers/base"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/agent/utils"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
func init() {
|
||||
handlers.RegisterEventHandleFunc(string(framework.PodEventName), NewResources)
|
||||
}
|
||||
|
||||
type ResourcesHandle struct {
|
||||
*base.BaseHandle
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
}
|
||||
|
||||
func NewResources(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, cgroupMgr cgroup.CgroupManager) framework.Handle {
|
||||
return &ResourcesHandle{
|
||||
BaseHandle: &base.BaseHandle{
|
||||
Name: string(features.ResourcesFeature),
|
||||
Config: config,
|
||||
Active: true,
|
||||
},
|
||||
cgroupMgr: cgroupMgr,
|
||||
}
|
||||
}
|
||||
|
||||
func (r *ResourcesHandle) Handle(event interface{}) error {
|
||||
podEvent, ok := event.(framework.PodEvent)
|
||||
if !ok {
|
||||
return fmt.Errorf("illegal pod event")
|
||||
}
|
||||
|
||||
if !allowedUseExtRes(podEvent.QoSLevel) {
|
||||
return nil
|
||||
}
|
||||
|
||||
resources := utilpod.CalculateExtendResources(podEvent.Pod)
|
||||
var errs []error
|
||||
// set container and pod level cgroup.
|
||||
for _, cr := range resources {
|
||||
cgroupPath, err := r.cgroupMgr.GetPodCgroupPath(podEvent.QoSClass, cr.CgroupSubSystem, podEvent.UID)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get pod cgroup", "pod", klog.KObj(podEvent.Pod), "subSystem", cr.CgroupSubSystem)
|
||||
errs = append(errs, err)
|
||||
}
|
||||
|
||||
filePath := path.Join(cgroupPath, cr.ContainerID, cr.SubPath)
|
||||
err = utils.UpdateFile(filePath, []byte(strconv.FormatInt(cr.Value, 10)))
|
||||
if os.IsNotExist(err) {
|
||||
klog.InfoS("Cgroup file not existed", "filePath", filePath)
|
||||
continue
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
errs = append(errs, err)
|
||||
klog.ErrorS(err, "Failed to set cgroup", "path", filePath, "pod", klog.KObj(podEvent.Pod))
|
||||
continue
|
||||
}
|
||||
klog.InfoS("Successfully set cpu and memory cgroup", "path", filePath, "pod", klog.KObj(podEvent.Pod))
|
||||
}
|
||||
return utilerrors.NewAggregate(errs)
|
||||
}
|
||||
|
||||
// allowedUseExtRes defines what qos levels can use extension resources,
|
||||
// currently only qos level QosLevelLS and QosLevelBE can use.
|
||||
func allowedUseExtRes(qosLevel int64) bool {
|
||||
return qosLevel <= 1
|
||||
}
|
||||
|
|
@ -0,0 +1,184 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package resources
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/resource"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/types"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/handlers/base"
|
||||
"volcano.sh/volcano/pkg/agent/features"
|
||||
"volcano.sh/volcano/pkg/agent/utils/cgroup"
|
||||
"volcano.sh/volcano/pkg/agent/utils/file"
|
||||
)
|
||||
|
||||
func TestResourcesHandle_Handle(t *testing.T) {
|
||||
tmpDir := t.TempDir()
|
||||
containerID1 := "65a6099d"
|
||||
containerID2 := "13b017b7"
|
||||
tests := []struct {
|
||||
name string
|
||||
cgroupMgr cgroup.CgroupManager
|
||||
event interface{}
|
||||
prepare func()
|
||||
post func() map[string]string
|
||||
wantErr bool
|
||||
wantVal map[string]string
|
||||
}{
|
||||
{
|
||||
name: "illegal pod event, return err",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
event: &framework.NodeResourceEvent{},
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "set correctly",
|
||||
cgroupMgr: cgroup.NewCgroupManager("cgroupfs", tmpDir, ""),
|
||||
event: framework.PodEvent{
|
||||
UID: "uid1",
|
||||
QoSLevel: -1,
|
||||
QoSClass: "Burstable",
|
||||
Pod: buildPodWithContainerID("p1", "uid1", containerID1, containerID2),
|
||||
},
|
||||
prepare: func() {
|
||||
prepare(t, tmpDir, "uid1", containerID1, containerID2)
|
||||
},
|
||||
post: func() map[string]string {
|
||||
return file.ReadBatchFromFile([]string{
|
||||
path.Join(tmpDir, "cpu/kubepods/burstable/poduid1/65a6099d/cpu.shares"),
|
||||
path.Join(tmpDir, "cpu/kubepods/burstable/poduid1/65a6099d/cpu.cfs_quota_us"),
|
||||
path.Join(tmpDir, "cpu/kubepods/burstable/poduid1/13b017b7/cpu.shares"),
|
||||
path.Join(tmpDir, "memory/kubepods/burstable/poduid1/13b017b7/memory.limit_in_bytes"),
|
||||
path.Join(tmpDir, "cpu/kubepods/burstable/poduid1/cpu.shares"),
|
||||
})
|
||||
},
|
||||
wantErr: false,
|
||||
wantVal: map[string]string{
|
||||
// container1
|
||||
path.Join(tmpDir, "cpu/kubepods/burstable/poduid1/65a6099d/cpu.shares"): "512",
|
||||
path.Join(tmpDir, "cpu/kubepods/burstable/poduid1/65a6099d/cpu.cfs_quota_us"): "200000",
|
||||
|
||||
// container2
|
||||
path.Join(tmpDir, "cpu/kubepods/burstable/poduid1/13b017b7/cpu.shares"): "1024",
|
||||
path.Join(tmpDir, "memory/kubepods/burstable/poduid1/13b017b7/memory.limit_in_bytes"): "10737418240",
|
||||
|
||||
// pod
|
||||
path.Join(tmpDir, "cpu/kubepods/burstable/poduid1/cpu.shares"): "1536",
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
r := &ResourcesHandle{
|
||||
BaseHandle: &base.BaseHandle{
|
||||
Name: string(features.ResourcesFeature),
|
||||
Config: nil,
|
||||
Active: true,
|
||||
},
|
||||
cgroupMgr: tt.cgroupMgr,
|
||||
}
|
||||
if tt.prepare != nil {
|
||||
tt.prepare()
|
||||
}
|
||||
if err := r.Handle(tt.event); (err != nil) != tt.wantErr {
|
||||
t.Errorf("Handle() error = %v, wantErr %v", err, tt.wantErr)
|
||||
}
|
||||
if tt.post != nil {
|
||||
assert.Equal(t, tt.wantVal, tt.post())
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func buildPodWithContainerID(name, uid, containerID1, containerID2 string) *v1.Pod {
|
||||
return &v1.Pod{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: name,
|
||||
UID: types.UID(uid),
|
||||
},
|
||||
Spec: v1.PodSpec{Containers: []v1.Container{
|
||||
{
|
||||
Name: "container-1",
|
||||
Resources: v1.ResourceRequirements{
|
||||
Limits: map[v1.ResourceName]resource.Quantity{
|
||||
"kubernetes.io/batch-cpu": *resource.NewQuantity(2000, resource.DecimalSI),
|
||||
},
|
||||
Requests: map[v1.ResourceName]resource.Quantity{
|
||||
"kubernetes.io/batch-cpu": *resource.NewQuantity(500, resource.DecimalSI),
|
||||
"kubernetes.io/batch-memory": *resource.NewQuantity(1024, resource.BinarySI),
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Name: "container-2",
|
||||
Resources: v1.ResourceRequirements{
|
||||
Limits: map[v1.ResourceName]resource.Quantity{
|
||||
// limit=10Gi
|
||||
"kubernetes.io/batch-memory": *resource.NewQuantity(1024*1024*1024*10, resource.BinarySI),
|
||||
},
|
||||
Requests: map[v1.ResourceName]resource.Quantity{
|
||||
"kubernetes.io/batch-cpu": *resource.NewQuantity(1000, resource.DecimalSI),
|
||||
"kubernetes.io/batch-memory": *resource.NewQuantity(1024*1024*1024*5, resource.BinarySI),
|
||||
},
|
||||
},
|
||||
},
|
||||
}},
|
||||
Status: v1.PodStatus{
|
||||
ContainerStatuses: []v1.ContainerStatus{
|
||||
{
|
||||
Name: "container-1",
|
||||
ContainerID: fmt.Sprintf("containerd://%s", containerID1),
|
||||
},
|
||||
{
|
||||
Name: "container-2",
|
||||
ContainerID: fmt.Sprintf("docker://%s", containerID2),
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func prepare(t *testing.T, tmpDir, podUID, containerID1, containerID2 string) {
|
||||
containers := []string{containerID1, containerID2}
|
||||
cgroupPaths := []string{"cpu.cfs_quota_us", "cpu.shares", "memory.limit_in_bytes"}
|
||||
subSystems := []string{"cpu", "memory"}
|
||||
for _, c := range containers {
|
||||
for _, ss := range subSystems {
|
||||
podDir := path.Join(tmpDir, ss, "kubepods", "burstable", "pod"+podUID)
|
||||
containerDir := path.Join(podDir, c)
|
||||
err := os.MkdirAll(containerDir, 0644)
|
||||
assert.NoError(t, err)
|
||||
for _, cgrouPath := range cgroupPaths {
|
||||
// create pod level cgroup.
|
||||
_, err := os.OpenFile(path.Join(podDir, cgrouPath), os.O_RDWR|os.O_CREATE, 0644)
|
||||
assert.NoError(t, err)
|
||||
// create container level cgroup.
|
||||
_, err = os.OpenFile(path.Join(containerDir, cgrouPath), os.O_RDWR|os.O_CREATE, 0644)
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,202 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package nodemonitor
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/util/wait"
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/probes"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/queue"
|
||||
"volcano.sh/volcano/pkg/agent/utils"
|
||||
"volcano.sh/volcano/pkg/agent/utils/eviction"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
"volcano.sh/volcano/pkg/metriccollect/local"
|
||||
"volcano.sh/volcano/pkg/resourceusage"
|
||||
)
|
||||
|
||||
func init() {
|
||||
probes.RegisterEventProbeFunc(string(framework.NodeMonitorEventName), NewMonitor)
|
||||
}
|
||||
|
||||
const (
|
||||
highUsageCountLimit = 6
|
||||
)
|
||||
|
||||
type monitor struct {
|
||||
sync.Mutex
|
||||
*config.Configuration
|
||||
policy.Interface
|
||||
cfgLock sync.RWMutex
|
||||
queue workqueue.RateLimitingInterface
|
||||
lowWatermark apis.Watermark
|
||||
highWatermark apis.Watermark
|
||||
// highUsageCountByResName is used to record whether resources usage are high.
|
||||
highUsageCountByResName map[v1.ResourceName]int
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
getPodsFunc utilpod.ActivePods
|
||||
usageGetter resourceusage.Getter
|
||||
}
|
||||
|
||||
func NewMonitor(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, workQueue workqueue.RateLimitingInterface) framework.Probe {
|
||||
evictor := eviction.NewEviction(config.GenericConfiguration.KubeClient, config.GenericConfiguration.KubeNodeName)
|
||||
return &monitor{
|
||||
Interface: policy.GetPolicyFunc(config.GenericConfiguration.OverSubscriptionPolicy)(config, mgr, evictor, queue.NewSqQueue(), local.CollectorName),
|
||||
queue: workQueue,
|
||||
getNodeFunc: config.GetNode,
|
||||
getPodsFunc: config.GetActivePods,
|
||||
lowWatermark: make(apis.Watermark),
|
||||
highWatermark: make(apis.Watermark),
|
||||
highUsageCountByResName: make(map[v1.ResourceName]int),
|
||||
usageGetter: resourceusage.NewUsageGetter(mgr, local.CollectorName),
|
||||
}
|
||||
}
|
||||
|
||||
func (m *monitor) ProbeName() string {
|
||||
return "NodePressureProbe"
|
||||
}
|
||||
|
||||
func (m *monitor) Run(stop <-chan struct{}) {
|
||||
klog.InfoS("Started nodePressure probe")
|
||||
go wait.Until(m.utilizationMonitoring, 10*time.Second, stop)
|
||||
go wait.Until(m.detect, 10*time.Second, stop)
|
||||
}
|
||||
|
||||
func (m *monitor) RefreshCfg(cfg *api.ColocationConfig) error {
|
||||
m.cfgLock.Lock()
|
||||
utils.SetEvictionWatermark(cfg, m.lowWatermark, m.highWatermark)
|
||||
m.cfgLock.Unlock()
|
||||
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
// reset historical statics
|
||||
// TODO: make this more fine-grained, only when new setting is a higher watermark should we reset.
|
||||
m.highUsageCountByResName = map[v1.ResourceName]int{}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *monitor) utilizationMonitoring() {
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
|
||||
node, err := m.getNodeFunc()
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Eviction: failed to get node")
|
||||
m.highUsageCountByResName = map[v1.ResourceName]int{}
|
||||
return
|
||||
}
|
||||
nodeCopy := node.DeepCopy()
|
||||
|
||||
// check if resource usage is high
|
||||
usage := m.usageGetter.UsagesByPercentage(nodeCopy)
|
||||
for _, res := range apis.OverSubscriptionResourceTypes {
|
||||
if m.isHighResourceUsageOnce(nodeCopy, apis.Resource(usage), res) {
|
||||
m.highUsageCountByResName[res]++
|
||||
} else {
|
||||
m.highUsageCountByResName[res] = 0
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (m *monitor) detect() {
|
||||
node, err := m.getNodeFunc()
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Eviction: failed to get node")
|
||||
return
|
||||
}
|
||||
nodeCopy := node.DeepCopy()
|
||||
|
||||
allResourcesAreLowUsage := true
|
||||
for _, res := range apis.OverSubscriptionResourceTypes {
|
||||
// Getting pod to be evicted should be executed in every resource for loop,
|
||||
// it's important because for every resource we should get the latest pods state.
|
||||
_, resList, err := utilnode.GetLatestPodsAndResList(nodeCopy, m.getPodsFunc, res)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get pods and resource list")
|
||||
return
|
||||
}
|
||||
if m.ShouldEvict(nodeCopy, res, resList, m.nodeHasPressure(res)) {
|
||||
event := framework.NodeMonitorEvent{
|
||||
TimeStamp: time.Now(),
|
||||
Resource: res,
|
||||
}
|
||||
klog.InfoS("Node pressure detected", "resource", res, "time", event.TimeStamp)
|
||||
m.queue.Add(event)
|
||||
}
|
||||
|
||||
usage := m.usageGetter.UsagesByPercentage(nodeCopy)
|
||||
if !m.isLowResourceUsageOnce(nodeCopy, apis.Resource(usage), res) {
|
||||
allResourcesAreLowUsage = false
|
||||
}
|
||||
}
|
||||
|
||||
// Only remove eviction annotation when all resources are low usage.
|
||||
if !allResourcesAreLowUsage {
|
||||
return
|
||||
}
|
||||
if err := m.RecoverSchedule(); err != nil {
|
||||
klog.ErrorS(err, "Failed to recover schedule")
|
||||
}
|
||||
}
|
||||
|
||||
func (m *monitor) isHighResourceUsageOnce(node *v1.Node, usage apis.Resource, resName v1.ResourceName) bool {
|
||||
m.cfgLock.RLock()
|
||||
defer m.cfgLock.RUnlock()
|
||||
//TODO: set in node config
|
||||
_, highWatermark, exists, err := utilnode.WatermarkAnnotationSetting(node)
|
||||
if !exists {
|
||||
return usage[resName] >= int64(m.highWatermark[resName])
|
||||
}
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get watermark in annotation")
|
||||
return usage[resName] >= int64(m.highWatermark[resName])
|
||||
}
|
||||
return usage[resName] >= highWatermark[resName]
|
||||
}
|
||||
|
||||
func (m *monitor) isLowResourceUsageOnce(node *v1.Node, usage apis.Resource, resName v1.ResourceName) bool {
|
||||
m.cfgLock.RLock()
|
||||
defer m.cfgLock.RUnlock()
|
||||
lowWatermark, _, exists, err := utilnode.WatermarkAnnotationSetting(node)
|
||||
if !exists {
|
||||
return usage[resName] <= int64(m.lowWatermark[resName])
|
||||
}
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get watermark in annotation")
|
||||
return usage[resName] <= int64(m.lowWatermark[resName])
|
||||
}
|
||||
return usage[resName] <= lowWatermark[resName]
|
||||
}
|
||||
|
||||
func (m *monitor) nodeHasPressure(resName v1.ResourceName) bool {
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
|
||||
return m.highUsageCountByResName[resName] >= highUsageCountLimit
|
||||
}
|
||||
|
|
@ -0,0 +1,151 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package nodemonitor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
fakeclientset "k8s.io/client-go/kubernetes/fake"
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy/extend"
|
||||
"volcano.sh/volcano/pkg/agent/utils/eviction"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/resourceusage"
|
||||
)
|
||||
|
||||
func makeNode() (*v1.Node, error) {
|
||||
return &v1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "test-node",
|
||||
Labels: map[string]string{
|
||||
apis.OverSubscriptionNodeLabelKey: "true",
|
||||
},
|
||||
},
|
||||
Spec: v1.NodeSpec{Taints: []v1.Taint{{
|
||||
Key: apis.PodEvictingKey,
|
||||
Effect: v1.TaintEffectNoSchedule,
|
||||
}}},
|
||||
}, nil
|
||||
}
|
||||
|
||||
func Test_monitor_detect(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
Configuration *config.Configuration
|
||||
policy func(cfg *config.Configuration, pods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface
|
||||
queue workqueue.RateLimitingInterface
|
||||
lowWatermark apis.Watermark
|
||||
highWatermark apis.Watermark
|
||||
highUsageCountByResName map[v1.ResourceName]int
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
getPodsFunc utilpod.ActivePods
|
||||
usageGetter resourceusage.Getter
|
||||
expectedNode func() *v1.Node
|
||||
expectedRes v1.ResourceName
|
||||
expectedLen int
|
||||
}{
|
||||
{
|
||||
name: "cpu in high usage with extend resource",
|
||||
highUsageCountByResName: map[v1.ResourceName]int{v1.ResourceCPU: 6},
|
||||
getNodeFunc: makeNode,
|
||||
getPodsFunc: func() ([]*v1.Pod, error) {
|
||||
return []*v1.Pod{}, nil
|
||||
},
|
||||
policy: func(cfg *config.Configuration, pods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface {
|
||||
return extend.NewExtendResource(cfg, nil, evictor, nil, "")
|
||||
},
|
||||
usageGetter: resourceusage.NewFakeResourceGetter(0, 0, 60, 60),
|
||||
expectedRes: v1.ResourceCPU,
|
||||
expectedLen: 1,
|
||||
},
|
||||
{
|
||||
name: "remove taint when use extend resource",
|
||||
highUsageCountByResName: map[v1.ResourceName]int{v1.ResourceCPU: 5},
|
||||
getNodeFunc: makeNode,
|
||||
getPodsFunc: func() ([]*v1.Pod, error) {
|
||||
return []*v1.Pod{}, nil
|
||||
},
|
||||
policy: func(cfg *config.Configuration, pods utilpod.ActivePods, evictor eviction.Eviction) policy.Interface {
|
||||
return extend.NewExtendResource(cfg, nil, evictor, nil, "")
|
||||
},
|
||||
usageGetter: resourceusage.NewFakeResourceGetter(0, 0, 20, 20),
|
||||
expectedRes: v1.ResourceCPU,
|
||||
expectedNode: func() *v1.Node {
|
||||
node, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
node.Spec.Taints = nil
|
||||
return node
|
||||
},
|
||||
expectedLen: 0,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
fakeNode, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
fakeClient := fakeclientset.NewSimpleClientset(fakeNode)
|
||||
cfg := &config.Configuration{GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
KubeClient: fakeClient,
|
||||
KubeNodeName: "test-node",
|
||||
NodeHasSynced: func() bool {
|
||||
return false
|
||||
},
|
||||
}}
|
||||
queue := workqueue.NewNamedRateLimitingQueue(nil, "test")
|
||||
m := &monitor{
|
||||
queue: queue,
|
||||
Configuration: cfg,
|
||||
Interface: tt.policy(cfg, nil, nil),
|
||||
highUsageCountByResName: tt.highUsageCountByResName,
|
||||
lowWatermark: map[v1.ResourceName]int{v1.ResourceCPU: 30, v1.ResourceMemory: 30},
|
||||
getNodeFunc: tt.getNodeFunc,
|
||||
getPodsFunc: tt.getPodsFunc,
|
||||
usageGetter: tt.usageGetter,
|
||||
}
|
||||
m.detect()
|
||||
assert.Equalf(t, tt.expectedLen, queue.Len(), "detect()")
|
||||
if queue.Len() != 0 {
|
||||
key, shutdown := queue.Get()
|
||||
if shutdown {
|
||||
t.Errorf("Unexpected: queue is shutdown")
|
||||
}
|
||||
event, ok := key.(framework.NodeMonitorEvent)
|
||||
if !ok {
|
||||
t.Errorf("Invalid event: %v", key)
|
||||
}
|
||||
assert.Equalf(t, tt.expectedRes, event.Resource, "detect()")
|
||||
}
|
||||
if tt.expectedNode != nil {
|
||||
node, err := fakeClient.CoreV1().Nodes().Get(context.TODO(), "test-node", metav1.GetOptions{})
|
||||
if err != nil {
|
||||
t.Errorf("Failed to get node, err: %v", err)
|
||||
}
|
||||
assert.Equalf(t, tt.expectedNode(), node, "detect()")
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,169 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package noderesources
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/util/sets"
|
||||
"k8s.io/apimachinery/pkg/util/wait"
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/probes"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/queue"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
"volcano.sh/volcano/pkg/metriccollect/local"
|
||||
)
|
||||
|
||||
func init() {
|
||||
probes.RegisterEventProbeFunc(string(framework.NodeResourcesEventName), NewCalculator)
|
||||
}
|
||||
|
||||
// historicalUsageCalculator is used to calculate the overSubscription
|
||||
type historicalUsageCalculator struct {
|
||||
sync.Mutex
|
||||
cfgLock sync.Mutex
|
||||
policy.Interface
|
||||
usages workqueue.RateLimitingInterface
|
||||
queue *queue.SqQueue
|
||||
resourceTypes sets.String
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
}
|
||||
|
||||
// NewCalculator return overSubscription reporter by algorithm
|
||||
func NewCalculator(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, workQueue workqueue.RateLimitingInterface) framework.Probe {
|
||||
sqQueue := queue.NewSqQueue()
|
||||
return &historicalUsageCalculator{
|
||||
Interface: policy.GetPolicyFunc(config.GenericConfiguration.OverSubscriptionPolicy)(config, mgr, nil, sqQueue, local.CollectorName),
|
||||
usages: workQueue,
|
||||
queue: sqQueue,
|
||||
resourceTypes: sets.NewString(),
|
||||
getNodeFunc: config.GetNode,
|
||||
}
|
||||
}
|
||||
|
||||
func (r *historicalUsageCalculator) Run(stop <-chan struct{}) {
|
||||
klog.InfoS("Started nodeResources probe")
|
||||
go wait.Until(r.CalOverSubscriptionResources, 10*time.Second, stop)
|
||||
go wait.Until(r.preProcess, 10*time.Second, stop)
|
||||
}
|
||||
|
||||
func (r *historicalUsageCalculator) ProbeName() string {
|
||||
return "NodeResourcesProbe"
|
||||
}
|
||||
|
||||
func (r *historicalUsageCalculator) RefreshCfg(cfg *api.ColocationConfig) error {
|
||||
r.cfgLock.Lock()
|
||||
defer r.cfgLock.Unlock()
|
||||
|
||||
if cfg == nil || cfg.OverSubscriptionConfig == nil || cfg.OverSubscriptionConfig.OverSubscriptionTypes == nil {
|
||||
return fmt.Errorf("nil colocation cfg")
|
||||
}
|
||||
// refresh overSubscription resource types.
|
||||
set := sets.NewString()
|
||||
typ := strings.Split(*cfg.OverSubscriptionConfig.OverSubscriptionTypes, ",")
|
||||
for _, resType := range typ {
|
||||
if resType == "" {
|
||||
continue
|
||||
}
|
||||
set.Insert(strings.TrimSpace(resType))
|
||||
}
|
||||
r.resourceTypes = set
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r *historicalUsageCalculator) preProcess() {
|
||||
node, err := r.getNodeFunc()
|
||||
if err != nil {
|
||||
klog.ErrorS(nil, "OverSubscription: failed to get node")
|
||||
return
|
||||
}
|
||||
nodeCopy := node.DeepCopy()
|
||||
if !r.SupportOverSubscription(nodeCopy) {
|
||||
return
|
||||
}
|
||||
|
||||
overSubRes := r.computeOverSubRes()
|
||||
if overSubRes == nil {
|
||||
return
|
||||
}
|
||||
customizationTypes := r.getOverSubscriptionTypes(nodeCopy)
|
||||
for _, resType := range apis.OverSubscriptionResourceTypes {
|
||||
if !customizationTypes[resType] {
|
||||
overSubRes[resType] = 0
|
||||
}
|
||||
}
|
||||
r.usages.Add(framework.NodeResourceEvent{MillCPU: overSubRes[v1.ResourceCPU], MemoryBytes: overSubRes[v1.ResourceMemory]})
|
||||
}
|
||||
|
||||
// computeOverSubRes calculate overSubscription resources
|
||||
func (r *historicalUsageCalculator) computeOverSubRes() apis.Resource {
|
||||
historicalUsages := r.queue.GetAll()
|
||||
if len(historicalUsages) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
overSubRes := make(apis.Resource)
|
||||
totalWeight := int64(0)
|
||||
initWeight := int64(1)
|
||||
for _, usage := range historicalUsages {
|
||||
totalWeight += initWeight
|
||||
for _, res := range apis.OverSubscriptionResourceTypes {
|
||||
overSubRes[res] = overSubRes[res] + usage[res]*initWeight
|
||||
}
|
||||
initWeight = initWeight * 2
|
||||
}
|
||||
for _, res := range apis.OverSubscriptionResourceTypes {
|
||||
overSubRes[res] = overSubRes[res] / totalWeight
|
||||
}
|
||||
return overSubRes
|
||||
}
|
||||
|
||||
func (r *historicalUsageCalculator) getOverSubscriptionTypes(node *v1.Node) map[v1.ResourceName]bool {
|
||||
r.cfgLock.Lock()
|
||||
defer r.cfgLock.Unlock()
|
||||
ret := make(map[v1.ResourceName]bool)
|
||||
for _, item := range r.resourceTypes.List() {
|
||||
ret[v1.ResourceName(item)] = true
|
||||
}
|
||||
|
||||
// be compatible with old api.
|
||||
value, exists := node.Annotations[apis.OverSubscriptionTypesKey]
|
||||
if !exists || value == "" {
|
||||
return ret
|
||||
}
|
||||
ret = make(map[v1.ResourceName]bool)
|
||||
typ := strings.Split(value, ",")
|
||||
for _, resType := range typ {
|
||||
if resType == "" {
|
||||
continue
|
||||
}
|
||||
ret[v1.ResourceName(strings.TrimSpace(resType))] = true
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
|
@ -0,0 +1,288 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package noderesources
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
"testing"
|
||||
|
||||
. "github.com/agiledragon/gomonkey/v2"
|
||||
"github.com/stretchr/testify/assert"
|
||||
v1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/api/resource"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/util/sets"
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
utilpointer "k8s.io/utils/pointer"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy/extend"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/queue"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
utiltesting "volcano.sh/volcano/pkg/agent/utils/testing"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
fakecollector "volcano.sh/volcano/pkg/metriccollect/testing"
|
||||
)
|
||||
|
||||
func makeNode() (*v1.Node, error) {
|
||||
return &v1.Node{
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Annotations: map[string]string{
|
||||
apis.OverSubscriptionTypesKey: "cpu",
|
||||
},
|
||||
Labels: map[string]string{
|
||||
apis.OverSubscriptionNodeLabelKey: "true",
|
||||
},
|
||||
},
|
||||
Status: v1.NodeStatus{Allocatable: map[v1.ResourceName]resource.Quantity{
|
||||
v1.ResourceCPU: *resource.NewMilliQuantity(3000, resource.DecimalSI),
|
||||
v1.ResourceMemory: *resource.NewQuantity(10000, resource.BinarySI),
|
||||
}},
|
||||
}, nil
|
||||
}
|
||||
|
||||
func Test_historicalUsageCalculator_utilizationMonitoring(t *testing.T) {
|
||||
dir, err := os.MkdirTemp("", "cpuManagerPolicy")
|
||||
assert.NoError(t, err)
|
||||
defer func() {
|
||||
err = os.RemoveAll(dir)
|
||||
assert.NoError(t, err)
|
||||
}()
|
||||
|
||||
cfg := &config.Configuration{
|
||||
GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
OverSubscriptionRatio: 60,
|
||||
},
|
||||
}
|
||||
|
||||
mgr, err := metriccollect.NewMetricCollectorManager(cfg, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
usages workqueue.RateLimitingInterface
|
||||
getPodsFunc utilpod.ActivePods
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
prepare func()
|
||||
expectRes []apis.Resource
|
||||
}{
|
||||
{
|
||||
name: "calculate using extend cpu&memory && cpu manager policy none",
|
||||
usages: workqueue.NewNamedRateLimitingQueue(nil, "calculator"),
|
||||
getNodeFunc: makeNode,
|
||||
prepare: func() {
|
||||
err = os.Setenv("KUBELET_ROOT_DIR", dir)
|
||||
assert.NoError(t, err)
|
||||
b := []byte(`{"policyName":"none","defaultCpuSet":"0-1","checksum":1636926438}`)
|
||||
err = os.WriteFile(path.Join(dir, "cpu_manager_state"), b, 0600)
|
||||
assert.NoError(t, err)
|
||||
},
|
||||
getPodsFunc: func() ([]*v1.Pod, error) {
|
||||
return []*v1.Pod{
|
||||
utiltesting.MakePod("online-1", 3, 2000, ""),
|
||||
utiltesting.MakePod("online-2", 2, 2000, ""),
|
||||
utiltesting.MakePodWithExtendResources("offline-1", 1000, 1000, "BE"),
|
||||
}, nil
|
||||
},
|
||||
expectRes: []apis.Resource{{v1.ResourceCPU: 1200, v1.ResourceMemory: 4800}},
|
||||
},
|
||||
{
|
||||
name: "calculate using extend cpu&memory && cpu manager policy static",
|
||||
usages: workqueue.NewNamedRateLimitingQueue(nil, "calculator"),
|
||||
getNodeFunc: makeNode,
|
||||
prepare: func() {
|
||||
err = os.Setenv("KUBELET_ROOT_DIR", dir)
|
||||
assert.NoError(t, err)
|
||||
b := []byte(`{"policyName":"static","defaultCpuSet":"0-1","checksum":1636926438}`)
|
||||
err = os.WriteFile(path.Join(dir, "cpu_manager_state"), b, 0600)
|
||||
assert.NoError(t, err)
|
||||
},
|
||||
getPodsFunc: func() ([]*v1.Pod, error) {
|
||||
return []*v1.Pod{
|
||||
utiltesting.MakePod("online-1", 1, 2000, ""),
|
||||
utiltesting.MakePod("online-2", 1, 2000, ""),
|
||||
utiltesting.MakePodWithExtendResources("offline-1", 1000, 1000, "BE"),
|
||||
}, nil
|
||||
},
|
||||
expectRes: []apis.Resource{{v1.ResourceCPU: 0, v1.ResourceMemory: 4800}},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
sqQueue := queue.NewSqQueue()
|
||||
|
||||
ApplyFunc(cfg.GetActivePods, func() ([]*v1.Pod, error) {
|
||||
return tt.getPodsFunc()
|
||||
})
|
||||
ApplyFunc(cfg.GetNode, func() (*v1.Node, error) {
|
||||
return tt.getNodeFunc()
|
||||
})
|
||||
r := &historicalUsageCalculator{
|
||||
// fake collector: cpu:1000m, memory:2000byte
|
||||
Interface: extend.NewExtendResource(cfg, mgr, nil, sqQueue, fakecollector.CollectorName),
|
||||
usages: tt.usages,
|
||||
queue: sqQueue,
|
||||
getNodeFunc: tt.getNodeFunc,
|
||||
}
|
||||
if tt.prepare != nil {
|
||||
tt.prepare()
|
||||
}
|
||||
r.CalOverSubscriptionResources()
|
||||
res := r.queue.GetAll()
|
||||
assert.Equalf(t, tt.expectRes, res, "utilizationMonitoring")
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func Test_historicalUsageCalculator_preProcess(t *testing.T) {
|
||||
queue := queue.NewSqQueue()
|
||||
queue.Enqueue(apis.Resource{
|
||||
v1.ResourceCPU: 1000,
|
||||
v1.ResourceMemory: 1500,
|
||||
})
|
||||
|
||||
queue.Enqueue(apis.Resource{
|
||||
v1.ResourceCPU: 1000,
|
||||
v1.ResourceMemory: 1500,
|
||||
})
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
getPodsFunc utilpod.ActivePods
|
||||
resourceTypes sets.String
|
||||
expectedResource framework.NodeResourceEvent
|
||||
}{
|
||||
{
|
||||
name: "default report cpu and memory",
|
||||
getNodeFunc: func() (*v1.Node, error) {
|
||||
node, err := makeNode()
|
||||
assert.NoError(t, err)
|
||||
node.Annotations = nil
|
||||
return node, nil
|
||||
},
|
||||
getPodsFunc: func() ([]*v1.Pod, error) {
|
||||
return []*v1.Pod{
|
||||
utiltesting.MakePod("online-1", 3, 2000, ""),
|
||||
utiltesting.MakePod("online-2", 2, 2000, ""),
|
||||
utiltesting.MakePod("offline-1", 2, 2000, "BE"),
|
||||
}, nil
|
||||
},
|
||||
expectedResource: framework.NodeResourceEvent{
|
||||
MillCPU: 1000,
|
||||
MemoryBytes: 1500,
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "only report cpu with node annotation specified",
|
||||
getNodeFunc: makeNode,
|
||||
getPodsFunc: func() ([]*v1.Pod, error) {
|
||||
return []*v1.Pod{
|
||||
utiltesting.MakePod("online-1", 3, 2000, ""),
|
||||
utiltesting.MakePod("online-2", 2, 2000, ""),
|
||||
utiltesting.MakePod("online-3", 2, 2000, ""),
|
||||
}, nil
|
||||
},
|
||||
expectedResource: framework.NodeResourceEvent{
|
||||
MillCPU: 1000,
|
||||
MemoryBytes: 0,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
cfg := &config.Configuration{
|
||||
GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
OverSubscriptionRatio: 60,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
r := &historicalUsageCalculator{
|
||||
Interface: extend.NewExtendResource(cfg, nil, nil, queue, ""),
|
||||
queue: queue,
|
||||
usages: workqueue.NewNamedRateLimitingQueue(nil, ""),
|
||||
getNodeFunc: tt.getNodeFunc,
|
||||
resourceTypes: sets.NewString("cpu", "memory"),
|
||||
}
|
||||
r.preProcess()
|
||||
usages, shutdown := r.usages.Get()
|
||||
if shutdown {
|
||||
t.Errorf("queue shutdown")
|
||||
}
|
||||
event, ok := usages.(framework.NodeResourceEvent)
|
||||
if !ok {
|
||||
t.Errorf("illegal node resource event")
|
||||
}
|
||||
assert.Equal(t, tt.expectedResource, event)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func Test_historicalUsageCalculator_RefreshCfg(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
resourceTypes sets.String
|
||||
cfg *api.ColocationConfig
|
||||
expectedResourceType sets.String
|
||||
wantErr assert.ErrorAssertionFunc
|
||||
}{
|
||||
{
|
||||
name: "return err",
|
||||
cfg: nil,
|
||||
resourceTypes: sets.NewString(),
|
||||
expectedResourceType: sets.NewString(),
|
||||
wantErr: assert.Error,
|
||||
},
|
||||
{
|
||||
name: "empty config",
|
||||
cfg: &api.ColocationConfig{OverSubscriptionConfig: &api.OverSubscription{OverSubscriptionTypes: utilpointer.String("")}},
|
||||
resourceTypes: sets.NewString(),
|
||||
expectedResourceType: sets.NewString(),
|
||||
wantErr: assert.NoError,
|
||||
},
|
||||
{
|
||||
name: "cpu",
|
||||
cfg: &api.ColocationConfig{OverSubscriptionConfig: &api.OverSubscription{OverSubscriptionTypes: utilpointer.String("cpu")}},
|
||||
resourceTypes: sets.NewString(),
|
||||
expectedResourceType: sets.NewString("cpu"),
|
||||
wantErr: assert.NoError,
|
||||
},
|
||||
{
|
||||
name: "cpu and memory",
|
||||
cfg: &api.ColocationConfig{OverSubscriptionConfig: &api.OverSubscription{OverSubscriptionTypes: utilpointer.String("cpu,memory")}},
|
||||
resourceTypes: sets.NewString(),
|
||||
expectedResourceType: sets.NewString("cpu", "memory"),
|
||||
wantErr: assert.NoError,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
r := &historicalUsageCalculator{
|
||||
resourceTypes: tt.resourceTypes,
|
||||
}
|
||||
tt.wantErr(t, r.RefreshCfg(tt.cfg), fmt.Sprintf("RefreshCfg(%v)", tt.cfg))
|
||||
assert.Equal(t, tt.expectedResourceType, r.resourceTypes)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,86 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package pods
|
||||
|
||||
import (
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/client-go/tools/cache"
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis/extension"
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/agent/events/probes"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
func init() {
|
||||
probes.RegisterEventProbeFunc(string(framework.PodEventName), NewPodProbe)
|
||||
}
|
||||
|
||||
type PodProbe struct {
|
||||
queue workqueue.RateLimitingInterface
|
||||
}
|
||||
|
||||
func NewPodProbe(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, queue workqueue.RateLimitingInterface) framework.Probe {
|
||||
podHandler := cache.ResourceEventHandlerFuncs{
|
||||
AddFunc: func(obj interface{}) { enqueuePod(obj, queue) },
|
||||
UpdateFunc: func(oldObj, newObj interface{}) { enqueuePod(newObj, queue) },
|
||||
DeleteFunc: func(obj interface{}) {},
|
||||
}
|
||||
config.InformerFactory.K8SInformerFactory.Core().V1().Pods().Informer().AddEventHandler(podHandler)
|
||||
return &PodProbe{
|
||||
queue: queue,
|
||||
}
|
||||
}
|
||||
|
||||
func (p *PodProbe) ProbeName() string {
|
||||
return "PodProbe"
|
||||
}
|
||||
|
||||
func (p *PodProbe) Run(stop <-chan struct{}) {
|
||||
klog.InfoS("Start pod probe")
|
||||
<-stop
|
||||
}
|
||||
|
||||
func (p *PodProbe) RefreshCfg(cfg *api.ColocationConfig) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func enqueuePod(obj interface{}, queue workqueue.RateLimitingInterface) {
|
||||
pod, ok := obj.(*corev1.Pod)
|
||||
if !ok {
|
||||
klog.ErrorS(nil, "Pod phase invoked with an invalid data struct", "obj", obj)
|
||||
return
|
||||
}
|
||||
|
||||
for _, podCondition := range pod.Status.Conditions {
|
||||
if podCondition.Type == corev1.PodReady && podCondition.Status == corev1.ConditionTrue && pod.DeletionTimestamp == nil {
|
||||
qosLevel := extension.GetQosLevel(pod)
|
||||
podEvent := framework.PodEvent{
|
||||
UID: pod.UID,
|
||||
QoSClass: pod.Status.QOSClass,
|
||||
QoSLevel: int64(qosLevel),
|
||||
Pod: pod,
|
||||
}
|
||||
klog.V(5).InfoS("Receive pod event", "pod", klog.KObj(pod))
|
||||
queue.Add(podEvent)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package probes
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/events/framework"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
var probeFuncs = map[string][]NewEventProbeFunc{}
|
||||
var mutex sync.Mutex
|
||||
|
||||
type NewEventProbeFunc = func(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, queue workqueue.RateLimitingInterface) framework.Probe
|
||||
|
||||
func RegisterEventProbeFunc(eventName string, f NewEventProbeFunc) {
|
||||
mutex.Lock()
|
||||
defer mutex.Unlock()
|
||||
|
||||
probeFuncs[eventName] = append(probeFuncs[eventName], f)
|
||||
}
|
||||
|
||||
func GetEventProbeFuncs() map[string][]NewEventProbeFunc {
|
||||
return probeFuncs
|
||||
}
|
||||
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package features
|
||||
|
||||
type Feature string
|
||||
|
||||
const (
|
||||
CPUQoSFeature Feature = "CPUQoS"
|
||||
CPUBurstFeature Feature = "CPUBurst"
|
||||
MemoryQoSFeature Feature = "MemoryQoS"
|
||||
NetworkQoSFeature Feature = "NetworkQoS"
|
||||
OverSubscriptionFeature Feature = "OverSubscription"
|
||||
EvictionFeature Feature = "Eviction"
|
||||
ResourcesFeature Feature = "Resources"
|
||||
)
|
||||
|
|
@ -0,0 +1,138 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package features
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"strings"
|
||||
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
agentutils "volcano.sh/volcano/pkg/agent/utils"
|
||||
"volcano.sh/volcano/pkg/agent/utils/version"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
var (
|
||||
// DefaultFeatureGate is a shared global FeatureGate.
|
||||
DefaultFeatureGate FeatureGate = &featureGate{}
|
||||
)
|
||||
|
||||
// FeatureGate indicates whether a given feature is enabled or not
|
||||
type FeatureGate interface {
|
||||
// Enabled returns true if the key is enabled.
|
||||
Enabled(key Feature, c *api.ColocationConfig) (bool, error)
|
||||
// Supported returns nil if the key is supported, return error with the reason why not supported
|
||||
Supported(key Feature, cfg *config.Configuration) error
|
||||
}
|
||||
|
||||
type featureGate struct{}
|
||||
|
||||
func (f *featureGate) Enabled(key Feature, c *api.ColocationConfig) (bool, error) {
|
||||
if c.NodeLabelConfig == nil {
|
||||
return false, fmt.Errorf("nil node label config")
|
||||
}
|
||||
nodeColocationEnabled := c.NodeLabelConfig.NodeColocationEnable != nil && *c.NodeLabelConfig.NodeColocationEnable
|
||||
nodeOverSubscriptionEnabled := c.NodeLabelConfig.NodeOverSubscriptionEnable != nil && *c.NodeLabelConfig.NodeOverSubscriptionEnable
|
||||
|
||||
switch key {
|
||||
case CPUQoSFeature:
|
||||
if c.CPUQosConfig == nil || c.CPUQosConfig.Enable == nil {
|
||||
return false, fmt.Errorf("nil cpu qos config")
|
||||
}
|
||||
return (nodeColocationEnabled || nodeOverSubscriptionEnabled) && *c.CPUQosConfig.Enable, nil
|
||||
|
||||
case CPUBurstFeature:
|
||||
if c.CPUBurstConfig == nil || c.CPUBurstConfig.Enable == nil {
|
||||
return false, fmt.Errorf("nil cpu burst config")
|
||||
}
|
||||
return (nodeColocationEnabled || nodeOverSubscriptionEnabled) && *c.CPUBurstConfig.Enable, nil
|
||||
|
||||
case MemoryQoSFeature:
|
||||
if c.MemoryQosConfig == nil || c.MemoryQosConfig.Enable == nil {
|
||||
return false, fmt.Errorf("nil memory qos config")
|
||||
}
|
||||
return (nodeColocationEnabled || nodeOverSubscriptionEnabled) && *c.MemoryQosConfig.Enable, nil
|
||||
|
||||
case NetworkQoSFeature:
|
||||
if c.NetworkQosConfig == nil || c.NetworkQosConfig.Enable == nil {
|
||||
return false, fmt.Errorf("nil memory qos config")
|
||||
}
|
||||
return (nodeColocationEnabled || nodeOverSubscriptionEnabled) && *c.NetworkQosConfig.Enable, nil
|
||||
|
||||
case OverSubscriptionFeature:
|
||||
if c.OverSubscriptionConfig == nil || c.OverSubscriptionConfig.Enable == nil {
|
||||
return false, fmt.Errorf("nil overSubscription config")
|
||||
}
|
||||
return nodeOverSubscriptionEnabled && *c.OverSubscriptionConfig.Enable, nil
|
||||
case EvictionFeature, ResourcesFeature:
|
||||
// Always return true because eviction manager need take care of all nodes.
|
||||
return true, nil
|
||||
default:
|
||||
return false, fmt.Errorf("unsupported feature %s", string(key))
|
||||
}
|
||||
}
|
||||
|
||||
type UnsupportedError string
|
||||
|
||||
func (s UnsupportedError) Error() string {
|
||||
return "unsupported feature: " + string(s)
|
||||
}
|
||||
|
||||
func IsUnsupportedError(err error) bool {
|
||||
_, ok := err.(UnsupportedError)
|
||||
return ok
|
||||
}
|
||||
|
||||
func (f *featureGate) Supported(feature Feature, cfg *config.Configuration) error {
|
||||
if !cfg.IsFeatureSupported(string(feature)) {
|
||||
return UnsupportedError(fmt.Sprintf("feature(%s) is not supprted by volcano-agent", string(feature)))
|
||||
}
|
||||
|
||||
switch feature {
|
||||
case NetworkQoSFeature:
|
||||
if err := CheckNodeSupportNetworkQoS(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func CheckNodeSupportNetworkQoS() error {
|
||||
return checkNodeOSSupportNetworkQoS()
|
||||
}
|
||||
|
||||
func checkNodeOSSupportNetworkQoS() error {
|
||||
osReleaseFile := strings.TrimSpace(os.Getenv(utils.HostOSReleasePathEnv))
|
||||
if osReleaseFile == "" {
|
||||
osReleaseFile = utils.DefaultNodeOSReleasePath
|
||||
}
|
||||
osRelease, err := agentutils.GetOSReleaseFromFile(osReleaseFile)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if osRelease.Name == utils.OpenEulerOSReleaseName && version.HigherOrEqual(osRelease.Version, utils.OpenEulerOSReleaseVersion) {
|
||||
return nil
|
||||
}
|
||||
klog.V(4).InfoS("os does not support network qos", "os-name", osRelease.Name, "os-version", osRelease.Version)
|
||||
return UnsupportedError(fmt.Sprintf("os(%s:%s) does not support network qos", osRelease.Name, osRelease.Version))
|
||||
}
|
||||
|
|
@ -0,0 +1,384 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package features
|
||||
|
||||
import (
|
||||
"os"
|
||||
"path"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
utilpointer "k8s.io/utils/pointer"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/config/api"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/networkqos/utils"
|
||||
)
|
||||
|
||||
func TestFeatureEnable(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
cfg *api.ColocationConfig
|
||||
expectedFeatures map[Feature]bool
|
||||
expectedErrors map[Feature]bool
|
||||
}{
|
||||
{
|
||||
name: "all-features-enabled",
|
||||
cfg: &api.ColocationConfig{
|
||||
NodeLabelConfig: &api.NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(true),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(true),
|
||||
},
|
||||
CPUQosConfig: &api.CPUQos{Enable: utilpointer.Bool(true)},
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(true)},
|
||||
MemoryQosConfig: &api.MemoryQos{Enable: utilpointer.Bool(true)},
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
},
|
||||
expectedFeatures: map[Feature]bool{
|
||||
CPUQoSFeature: true,
|
||||
CPUBurstFeature: true,
|
||||
MemoryQoSFeature: true,
|
||||
NetworkQoSFeature: true,
|
||||
OverSubscriptionFeature: true,
|
||||
},
|
||||
expectedErrors: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
name: "node-colocation-disabled && node-OverSubscription-disabled",
|
||||
cfg: &api.ColocationConfig{
|
||||
NodeLabelConfig: &api.NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(false),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(false),
|
||||
},
|
||||
CPUQosConfig: &api.CPUQos{Enable: utilpointer.Bool(true)},
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(true)},
|
||||
MemoryQosConfig: &api.MemoryQos{Enable: utilpointer.Bool(true)},
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
},
|
||||
expectedFeatures: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
expectedErrors: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
name: "node-colocation-enabled && node-OverSubscription-disabled",
|
||||
cfg: &api.ColocationConfig{
|
||||
NodeLabelConfig: &api.NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(true),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(false),
|
||||
},
|
||||
CPUQosConfig: &api.CPUQos{Enable: utilpointer.Bool(true)},
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(true)},
|
||||
MemoryQosConfig: &api.MemoryQos{Enable: utilpointer.Bool(true)},
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
},
|
||||
expectedFeatures: map[Feature]bool{
|
||||
CPUQoSFeature: true,
|
||||
CPUBurstFeature: true,
|
||||
MemoryQoSFeature: true,
|
||||
NetworkQoSFeature: true,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
expectedErrors: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
name: "node-colocation-disabled && node-OverSubscription-enabled",
|
||||
cfg: &api.ColocationConfig{
|
||||
NodeLabelConfig: &api.NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(false),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(true),
|
||||
},
|
||||
CPUQosConfig: &api.CPUQos{Enable: utilpointer.Bool(true)},
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(true)},
|
||||
MemoryQosConfig: &api.MemoryQos{Enable: utilpointer.Bool(true)},
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
},
|
||||
expectedFeatures: map[Feature]bool{
|
||||
CPUQoSFeature: true,
|
||||
CPUBurstFeature: true,
|
||||
MemoryQoSFeature: true,
|
||||
NetworkQoSFeature: true,
|
||||
OverSubscriptionFeature: true,
|
||||
},
|
||||
expectedErrors: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
name: "cpu-qos-disabled && memory-qos-disabled && network-qos-disabled",
|
||||
cfg: &api.ColocationConfig{
|
||||
NodeLabelConfig: &api.NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(true),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(false),
|
||||
},
|
||||
CPUQosConfig: &api.CPUQos{Enable: utilpointer.Bool(false)},
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(true)},
|
||||
MemoryQosConfig: &api.MemoryQos{Enable: utilpointer.Bool(false)},
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(false),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
},
|
||||
expectedFeatures: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: true,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
expectedErrors: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
name: "cpu-burst-disabled && overSubscription-disabled",
|
||||
cfg: &api.ColocationConfig{
|
||||
NodeLabelConfig: &api.NodeLabelConfig{
|
||||
NodeColocationEnable: utilpointer.Bool(false),
|
||||
NodeOverSubscriptionEnable: utilpointer.Bool(true),
|
||||
},
|
||||
CPUQosConfig: &api.CPUQos{Enable: utilpointer.Bool(true)},
|
||||
CPUBurstConfig: &api.CPUBurst{Enable: utilpointer.Bool(false)},
|
||||
MemoryQosConfig: &api.MemoryQos{Enable: utilpointer.Bool(true)},
|
||||
NetworkQosConfig: &api.NetworkQos{
|
||||
Enable: utilpointer.Bool(true),
|
||||
},
|
||||
OverSubscriptionConfig: &api.OverSubscription{
|
||||
Enable: utilpointer.Bool(false),
|
||||
},
|
||||
},
|
||||
expectedFeatures: map[Feature]bool{
|
||||
CPUQoSFeature: true,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: true,
|
||||
NetworkQoSFeature: true,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
expectedErrors: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
for feature, expectedEnabled := range tt.expectedFeatures {
|
||||
actualEnabled, actualErr := DefaultFeatureGate.Enabled(feature, tt.cfg)
|
||||
assert.Equal(t, expectedEnabled, actualEnabled, tt.name, feature)
|
||||
assert.Equal(t, tt.expectedErrors[feature], actualErr != nil, tt.name, feature)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestFeatureSupport(t *testing.T) {
|
||||
dir, err := os.MkdirTemp("/tmp", "MkdirTemp")
|
||||
defer func() {
|
||||
err = os.RemoveAll(dir)
|
||||
if err != nil {
|
||||
t.Errorf("remove dir(%s) failed: %v", dir, err)
|
||||
}
|
||||
assert.Equal(t, err == nil, true)
|
||||
}()
|
||||
assert.Equal(t, err == nil, true)
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
configuration *config.Configuration
|
||||
expectedErrors map[Feature]bool
|
||||
}{
|
||||
{
|
||||
name: "all-features-enabled",
|
||||
configuration: &config.Configuration{
|
||||
GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
SupportedFeatures: []string{"*"},
|
||||
},
|
||||
},
|
||||
expectedErrors: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
name: "all-features-enabled && support networkqos && support memoryqos",
|
||||
configuration: &config.Configuration{
|
||||
GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
SupportedFeatures: []string{"NetworkQoS", "MemoryQoS"},
|
||||
},
|
||||
},
|
||||
expectedErrors: map[Feature]bool{
|
||||
CPUQoSFeature: true,
|
||||
CPUBurstFeature: true,
|
||||
MemoryQoSFeature: false,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: true,
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
name: "all-features-enabled && not support memoryqos",
|
||||
configuration: &config.Configuration{
|
||||
GenericConfiguration: &config.VolcanoAgentConfiguration{
|
||||
SupportedFeatures: []string{"*", "-MemoryQoS"},
|
||||
},
|
||||
},
|
||||
expectedErrors: map[Feature]bool{
|
||||
CPUQoSFeature: false,
|
||||
CPUBurstFeature: false,
|
||||
MemoryQoSFeature: true,
|
||||
NetworkQoSFeature: false,
|
||||
OverSubscriptionFeature: false,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
tmpFile := path.Join(dir, "os-release")
|
||||
if err = os.WriteFile(tmpFile, []byte(openEulerOS), 0660); err != nil {
|
||||
assert.Equal(t, nil, err)
|
||||
}
|
||||
if err = os.Setenv(utils.HostOSReleasePathEnv, tmpFile); err != nil {
|
||||
assert.Equal(t, nil, err)
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
for feature, expectedErr := range tt.expectedErrors {
|
||||
actualErr := DefaultFeatureGate.Supported(feature, tt.configuration)
|
||||
assert.Equal(t, expectedErr, actualErr != nil, tt.name, feature)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
var openEulerOS = `
|
||||
NAME="openEuler"
|
||||
VERSION="22.03 (LTS-SP2)"
|
||||
ID="openEuler"
|
||||
VERSION_ID="22.03"
|
||||
PRETTY_NAME="openEuler 22.03 (LTS-SP2)"
|
||||
ANSI_COLOR="0;31"
|
||||
`
|
||||
|
||||
var ubuntuOS = `
|
||||
NAME="Ubuntu"
|
||||
VERSION="16.04.5 LTS (Xenial Xerus)"
|
||||
ID=ubuntu
|
||||
ID_LIKE=debian
|
||||
PRETTY_NAME="Ubuntu 16.04.5 LTS"
|
||||
VERSION_ID="16.04"
|
||||
`
|
||||
|
||||
func TestCheckNodeSupportNetworkQoS(t *testing.T) {
|
||||
dir, err := os.MkdirTemp("/tmp", "MkdirTemp")
|
||||
defer func() {
|
||||
err = os.RemoveAll(dir)
|
||||
if err != nil {
|
||||
t.Errorf("remove dir(%s) failed: %v", dir, err)
|
||||
}
|
||||
assert.Equal(t, err == nil, true)
|
||||
}()
|
||||
assert.Equal(t, err == nil, true)
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
releaseContent string
|
||||
expectedError bool
|
||||
}{
|
||||
{
|
||||
name: "openEuler 22.03 SP2",
|
||||
releaseContent: openEulerOS,
|
||||
expectedError: false,
|
||||
},
|
||||
{
|
||||
name: "ubuntu && x86",
|
||||
releaseContent: ubuntuOS,
|
||||
expectedError: true,
|
||||
},
|
||||
}
|
||||
for _, tc := range testCases {
|
||||
tmpFile := path.Join(dir, "os-release")
|
||||
if err = os.WriteFile(tmpFile, []byte(tc.releaseContent), 0660); err != nil {
|
||||
assert.Equal(t, nil, err)
|
||||
}
|
||||
if err = os.Setenv(utils.HostOSReleasePathEnv, tmpFile); err != nil {
|
||||
assert.Equal(t, nil, err)
|
||||
}
|
||||
actualErr := CheckNodeSupportNetworkQoS()
|
||||
assert.Equal(t, tc.expectedError, actualErr != nil, tc.name)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,57 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package healthcheck
|
||||
|
||||
import (
|
||||
"net/http"
|
||||
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/networkqos"
|
||||
)
|
||||
|
||||
type HealthChecker interface {
|
||||
HealthCheck(w http.ResponseWriter, r *http.Request)
|
||||
}
|
||||
|
||||
type healthChecker struct {
|
||||
networkQoSMgr networkqos.NetworkQoSManager
|
||||
}
|
||||
|
||||
func NewHealthChecker(networkQoSMgr networkqos.NetworkQoSManager) HealthChecker {
|
||||
return &healthChecker{
|
||||
networkQoSMgr: networkQoSMgr,
|
||||
}
|
||||
}
|
||||
|
||||
func (c *healthChecker) HealthCheck(w http.ResponseWriter, r *http.Request) {
|
||||
if err := c.networkQoSMgr.HealthCheck(); err != nil {
|
||||
w.WriteHeader(http.StatusInternalServerError)
|
||||
if _, writeErr := w.Write([]byte(err.Error())); writeErr != nil {
|
||||
klog.ErrorS(writeErr, "Failed to check network qos")
|
||||
}
|
||||
klog.ErrorS(err, "Failed to check volcano-agent")
|
||||
return
|
||||
}
|
||||
|
||||
w.WriteHeader(http.StatusOK)
|
||||
if _, writeErr := w.Write([]byte(`ok`)); writeErr != nil {
|
||||
klog.ErrorS(writeErr, "Failed to write ok to response")
|
||||
} else {
|
||||
klog.V(3).InfoS("successfully checking health of volcano-agent ")
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package metrics
|
||||
|
||||
import (
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/client_golang/prometheus/promauto"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
)
|
||||
|
||||
const (
|
||||
subSystem = "volcano_agent"
|
||||
)
|
||||
|
||||
var overSubscriptionResourceQuantity = promauto.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Subsystem: subSystem,
|
||||
Name: "oversubscription_resource_quantity",
|
||||
Help: "The reported overSubscription resource quantity",
|
||||
},
|
||||
[]string{"node", "resource"},
|
||||
)
|
||||
|
||||
// UpdateOverSubscriptionResourceQuantity update node overSubscription resource by resource name.
|
||||
func UpdateOverSubscriptionResourceQuantity(nodeName string, resources apis.Resource) {
|
||||
for resName, quantity := range resources {
|
||||
overSubscriptionResourceQuantity.WithLabelValues(nodeName, string(resName)).Set(float64(quantity))
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,152 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package extend
|
||||
|
||||
import (
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/policy"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/queue"
|
||||
"volcano.sh/volcano/pkg/agent/utils/eviction"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
"volcano.sh/volcano/pkg/resourceusage"
|
||||
)
|
||||
|
||||
func init() {
|
||||
policy.RegistryPolicy(string(ExtendResource), NewExtendResource)
|
||||
}
|
||||
|
||||
const ExtendResource policy.Name = "extend"
|
||||
|
||||
type extendResource struct {
|
||||
config *config.Configuration
|
||||
getPodsFunc utilpod.ActivePods
|
||||
getNodeFunc utilnode.ActiveNode
|
||||
evictor eviction.Eviction
|
||||
queue *queue.SqQueue
|
||||
usageGetter resourceusage.Getter
|
||||
ratio int
|
||||
}
|
||||
|
||||
func NewExtendResource(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, evictor eviction.Eviction, queue *queue.SqQueue, collectorName string) policy.Interface {
|
||||
return &extendResource{
|
||||
config: config,
|
||||
getPodsFunc: config.GetActivePods,
|
||||
getNodeFunc: config.GetNode,
|
||||
evictor: evictor,
|
||||
queue: queue,
|
||||
usageGetter: resourceusage.NewUsageGetter(mgr, collectorName),
|
||||
ratio: config.GenericConfiguration.OverSubscriptionRatio,
|
||||
}
|
||||
}
|
||||
|
||||
func (e *extendResource) Name() string {
|
||||
return string(ExtendResource)
|
||||
}
|
||||
|
||||
func (e *extendResource) SupportOverSubscription(node *corev1.Node) bool {
|
||||
return utilnode.IsNodeSupportOverSubscription(node)
|
||||
}
|
||||
|
||||
func (e *extendResource) ShouldEvict(node *corev1.Node, resName corev1.ResourceName, resList *utilnode.ResourceList, hasPressure bool) bool {
|
||||
return utilnode.IsNodeSupportOverSubscription(node) && hasPressure
|
||||
}
|
||||
|
||||
func (e *extendResource) ShouldUpdateOverSubscription(node *corev1.Node, resource apis.Resource) bool {
|
||||
currentOverSubscription := utilnode.GetNodeStatusOverSubscription(node)
|
||||
return policy.ShouldUpdateNodeOverSubscription(currentOverSubscription, resource)
|
||||
}
|
||||
|
||||
func (e *extendResource) UpdateOverSubscription(resource apis.Resource) error {
|
||||
return utilnode.UpdateNodeExtendResource(e.config, resource)
|
||||
}
|
||||
|
||||
func (e *extendResource) Cleanup() error {
|
||||
if err := utilnode.DeleteNodeOverSoldStatus(e.config); err != nil {
|
||||
klog.ErrorS(err, "Failed to reset overSubscription info")
|
||||
return err
|
||||
}
|
||||
klog.InfoS("Successfully reset overSubscription info")
|
||||
if err := policy.EvictPods(&policy.EvictionCtx{
|
||||
Configuration: e.config,
|
||||
Eviction: e.evictor,
|
||||
GracePeriodOverride: 0,
|
||||
EvictMsg: "Evict offline pod due to node overSubscription is turned off",
|
||||
GetPodsFunc: e.getPodsFunc,
|
||||
Filter: utilnode.UseExtendResource,
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *extendResource) DisableSchedule() error {
|
||||
return utilnode.DisableSchedule(e.config)
|
||||
}
|
||||
|
||||
func (e *extendResource) RecoverSchedule() error {
|
||||
return utilnode.RecoverSchedule(e.config)
|
||||
}
|
||||
|
||||
func (e *extendResource) CalOverSubscriptionResources() {
|
||||
node, err := e.getNodeFunc()
|
||||
if err != nil {
|
||||
klog.ErrorS(nil, "overSubscription: failed to get node")
|
||||
return
|
||||
}
|
||||
nodeCopy := node.DeepCopy()
|
||||
|
||||
if !e.SupportOverSubscription(nodeCopy) {
|
||||
return
|
||||
}
|
||||
|
||||
pods, err := e.getPodsFunc()
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get pods")
|
||||
return
|
||||
}
|
||||
|
||||
includeGuaranteedPods := utilpod.IncludeGuaranteedPods()
|
||||
currentUsage := e.usageGetter.UsagesByValue(includeGuaranteedPods)
|
||||
overSubscriptionRes := make(apis.Resource)
|
||||
|
||||
for _, resType := range apis.OverSubscriptionResourceTypes {
|
||||
total := int64(0)
|
||||
switch resType {
|
||||
case corev1.ResourceCPU:
|
||||
total = node.Status.Allocatable.Cpu().MilliValue() - utilpod.GuaranteedPodsCPURequest(pods)
|
||||
case corev1.ResourceMemory:
|
||||
total = node.Status.Allocatable.Memory().Value()
|
||||
default:
|
||||
klog.InfoS("overSubscription: reporter does not support resource", "resourceType", resType)
|
||||
}
|
||||
|
||||
if total >= currentUsage[resType] {
|
||||
overSubscriptionRes[resType] = (total - currentUsage[resType]) * int64(e.ratio) / 100
|
||||
} else {
|
||||
overSubscriptionRes[resType] = 0
|
||||
}
|
||||
|
||||
klog.V(4).InfoS("overSubscription:", "resourceType", resType, "total", total, "usage", currentUsage[resType], "delta", overSubscriptionRes[resType])
|
||||
}
|
||||
e.queue.Enqueue(overSubscriptionRes)
|
||||
}
|
||||
|
|
@ -0,0 +1,159 @@
|
|||
/*
|
||||
Copyright 2024 The Volcano Authors.
|
||||
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
*/
|
||||
|
||||
package policy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/klog/v2"
|
||||
|
||||
"volcano.sh/volcano/pkg/agent/apis"
|
||||
"volcano.sh/volcano/pkg/agent/oversubscription/queue"
|
||||
"volcano.sh/volcano/pkg/agent/utils/eviction"
|
||||
utilnode "volcano.sh/volcano/pkg/agent/utils/node"
|
||||
utilpod "volcano.sh/volcano/pkg/agent/utils/pod"
|
||||
"volcano.sh/volcano/pkg/config"
|
||||
"volcano.sh/volcano/pkg/metriccollect"
|
||||
)
|
||||
|
||||
var (
|
||||
lock sync.Mutex
|
||||
policyMap = make(map[string]PolicyFunc)
|
||||
)
|
||||
|
||||
const overSubscriptionChangeStep = 0.1
|
||||
|
||||
type PolicyFunc func(config *config.Configuration, mgr *metriccollect.MetricCollectorManager, evictor eviction.Eviction, queue *queue.SqQueue, collectorName string) Interface
|
||||
|
||||
type Name string
|
||||
|
||||
// Interface defines overSubscription resource policy, support overSubscription resource patch annotation currently.
|
||||
// You can register your own overSubscription policy like patch overSubscription resource on node.Allocatable.
|
||||
type Interface interface {
|
||||
// Name is the policy name.
|
||||
Name() string
|
||||
// SupportOverSubscription return whether node support over subscription.
|
||||
SupportOverSubscription(node *corev1.Node) bool
|
||||
// ShouldEvict return whether we should evict low priority pods.
|
||||
ShouldEvict(node *corev1.Node, resName corev1.ResourceName, resList *utilnode.ResourceList, hasPressure bool) bool
|
||||
// CalOverSubscriptionResources calculate overSubscription resources.
|
||||
CalOverSubscriptionResources()
|
||||
// ShouldUpdateOverSubscription return whether new overSubscription resources should be patched.
|
||||
ShouldUpdateOverSubscription(node *corev1.Node, resource apis.Resource) bool
|
||||
// UpdateOverSubscription will update overSubscription resource to node.
|
||||
UpdateOverSubscription(resource apis.Resource) error
|
||||
// Cleanup reset overSubscription label and evict low priority pods when turn off overSubscription.
|
||||
Cleanup() error
|
||||
// DisableSchedule disable schedule.
|
||||
DisableSchedule() error
|
||||
// RecoverSchedule recover schedule.
|
||||
RecoverSchedule() error
|
||||
}
|
||||
|
||||
func RegistryPolicy(name string, policyFunc PolicyFunc) {
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
|
||||
if _, exist := policyMap[name]; exist {
|
||||
klog.ErrorS(nil, "Policy has already been registered", "name", name)
|
||||
return
|
||||
}
|
||||
policyMap[name] = policyFunc
|
||||
}
|
||||
|
||||
func GetPolicyFunc(name string) PolicyFunc {
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
|
||||
fn, exist := policyMap[name]
|
||||
if !exist {
|
||||
klog.Fatalf("Policy %s not registered", name)
|
||||
}
|
||||
return fn
|
||||
}
|
||||
|
||||
type EvictionCtx struct {
|
||||
*config.Configuration
|
||||
eviction.Eviction
|
||||
GracePeriodOverride int64
|
||||
EvictMsg string
|
||||
GetPodsFunc utilpod.ActivePods
|
||||
Filter func(resName corev1.ResourceName, resList *utilnode.ResourceList) bool
|
||||
}
|
||||
|
||||
func EvictPods(ctx *EvictionCtx) error {
|
||||
evict := func(node *corev1.Node) (bool, error) {
|
||||
keepGoing := false
|
||||
for _, res := range apis.OverSubscriptionResourceTypes {
|
||||
// Getting pod to be evicted should be executed in every resource for loop,
|
||||
// it's important because for every resource we should get the latest pods state.
|
||||
preemptablePods, resList, err := utilnode.GetLatestPodsAndResList(node, ctx.GetPodsFunc, res)
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get pods and resource list")
|
||||
return true, err
|
||||
}
|
||||
|
||||
if !ctx.Filter(res, resList) {
|
||||
continue
|
||||
}
|
||||
for _, pod := range preemptablePods {
|
||||
klog.InfoS("Try to evict pod", "pod", klog.KObj(pod))
|
||||
if ctx.Evict(context.TODO(), pod, ctx.GenericConfiguration.Recorder, ctx.GracePeriodOverride, ctx.EvictMsg) {
|
||||
keepGoing = true
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
return keepGoing, nil
|
||||
}
|
||||
|
||||
for {
|
||||
node, err := ctx.GetNode()
|
||||
if err != nil {
|
||||
klog.ErrorS(err, "Failed to get node and pods")
|
||||
return err
|
||||
}
|
||||
|
||||
keepGoing, err := evict(node)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if !keepGoing {
|
||||
break
|
||||
}
|
||||
}
|
||||
klog.InfoS("Successfully cleaned up resources when turn off oversubscription")
|
||||
return nil
|
||||
}
|
||||
|
||||
func ShouldUpdateNodeOverSubscription(current, new apis.Resource) bool {
|
||||
update := false
|
||||
for _, res := range apis.OverSubscriptionResourceTypes {
|
||||
delta := new[res] - current[res]
|
||||
if delta < 0 {
|
||||
delta = -delta
|
||||
}
|
||||
|
||||
if float64(delta)/float64(current[res]) > overSubscriptionChangeStep {
|
||||
update = true
|
||||
break
|
||||
}
|
||||
}
|
||||
return update
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue