From 833244628a78b6b21f0ef232e288d20829438fab Mon Sep 17 00:00:00 2001 From: Jorge Turrado Ferrero Date: Mon, 11 Nov 2024 20:41:10 +0100 Subject: [PATCH 01/15] fix: Paused ScaledObject count is reported correctly after operator restart (#6322) Signed-off-by: Jorge Turrado --- CHANGELOG.md | 2 +- controllers/keda/scaledobject_controller.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index de0b5ebec20..dbba7b57a05 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -72,7 +72,7 @@ Here is an overview of all new **experimental** features: ### Fixes -- TODO ([#XXX](https://github.com/kedacore/keda/issues/XXX)) +- **General**: Paused ScaledObject count is reported correctly after operator restart ([#6321](https://github.com/kedacore/keda/issues/6321)) ### Deprecations diff --git a/controllers/keda/scaledobject_controller.go b/controllers/keda/scaledobject_controller.go index ff6194c4ea7..c480dc380c4 100755 --- a/controllers/keda/scaledobject_controller.go +++ b/controllers/keda/scaledobject_controller.go @@ -209,6 +209,8 @@ func (r *ScaledObjectReconciler) Reconcile(ctx context.Context, req ctrl.Request conditions.SetFallbackCondition(metav1.ConditionFalse, "NoFallbackFound", "No fallbacks are active on this scaled object") } + metricscollector.RecordScaledObjectPaused(scaledObject.Namespace, scaledObject.Name, conditions.GetPausedCondition().Status == metav1.ConditionTrue) + if err := kedastatus.SetStatusConditions(ctx, r.Client, reqLogger, scaledObject, &conditions); err != nil { r.EventEmitter.Emit(scaledObject, req.NamespacedName.Namespace, corev1.EventTypeWarning, eventingv1alpha1.ScaledObjectFailedType, eventreason.ScaledObjectUpdateFailed, err.Error()) return ctrl.Result{}, err @@ -246,12 +248,10 @@ func (r *ScaledObjectReconciler) reconcileScaledObject(ctx context.Context, logg return msg, err } conditions.SetPausedCondition(metav1.ConditionTrue, kedav1alpha1.ScaledObjectConditionPausedReason, msg) - metricscollector.RecordScaledObjectPaused(scaledObject.Namespace, scaledObject.Name, true) return msg, nil } } else if conditions.GetPausedCondition().Status == metav1.ConditionTrue { conditions.SetPausedCondition(metav1.ConditionFalse, "ScaledObjectUnpaused", "pause annotation removed for ScaledObject") - metricscollector.RecordScaledObjectPaused(scaledObject.Namespace, scaledObject.Name, false) } // Check scale target Name is specified From 8b4ccdb47e7ee8b798476f2b9f40ff03c53dbc6a Mon Sep 17 00:00:00 2001 From: Omer Aplatony Date: Thu, 21 Nov 2024 10:41:57 +0200 Subject: [PATCH 02/15] fix: ensure consistent JSON log format for automaxprocs (#6335) * fix: ensure consistent JSON log format for automaxprocs Signed-off-by: Omer Aplatony * moved to Unreleased Signed-off-by: Omer Aplatony --------- Signed-off-by: Omer Aplatony --- CHANGELOG.md | 1 + cmd/adapter/main.go | 7 ++++++- cmd/operator/main.go | 8 +++++++- cmd/webhooks/main.go | 7 ++++++- pkg/util/maxprocs.go | 34 ++++++++++++++++++++++++++++++++++ 5 files changed, 54 insertions(+), 3 deletions(-) create mode 100644 pkg/util/maxprocs.go diff --git a/CHANGELOG.md b/CHANGELOG.md index dbba7b57a05..df2a1987770 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -72,6 +72,7 @@ Here is an overview of all new **experimental** features: ### Fixes +- **General**: Centralize and improve automaxprocs configuration with proper structured logging ([#5970](https://github.com/kedacore/keda/issues/5970)) - **General**: Paused ScaledObject count is reported correctly after operator restart ([#6321](https://github.com/kedacore/keda/issues/6321)) ### Deprecations diff --git a/cmd/adapter/main.go b/cmd/adapter/main.go index b4ecf6781bb..8444e51dfac 100644 --- a/cmd/adapter/main.go +++ b/cmd/adapter/main.go @@ -26,7 +26,6 @@ import ( grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/collectors" - _ "go.uber.org/automaxprocs" appsv1 "k8s.io/api/apps/v1" apimetrics "k8s.io/apiserver/pkg/endpoints/metrics" "k8s.io/client-go/kubernetes/scheme" @@ -257,6 +256,12 @@ func main() { return } + err = kedautil.ConfigureMaxProcs(logger) + if err != nil { + logger.Error(err, "failed to set max procs") + return + } + kedaProvider, err := cmd.makeProvider(ctx) if err != nil { logger.Error(err, "making provider") diff --git a/cmd/operator/main.go b/cmd/operator/main.go index 16f6899d230..dd1dc656f28 100644 --- a/cmd/operator/main.go +++ b/cmd/operator/main.go @@ -22,7 +22,6 @@ import ( "time" "github.com/spf13/pflag" - _ "go.uber.org/automaxprocs" apimachineryruntime "k8s.io/apimachinery/pkg/runtime" utilruntime "k8s.io/apimachinery/pkg/util/runtime" kubeinformers "k8s.io/client-go/informers" @@ -115,6 +114,13 @@ func main() { ctrl.SetLogger(zap.New(zap.UseFlagOptions(&opts))) ctx := ctrl.SetupSignalHandler() + + err := kedautil.ConfigureMaxProcs(setupLog) + if err != nil { + setupLog.Error(err, "failed to set max procs") + os.Exit(1) + } + namespaces, err := kedautil.GetWatchNamespaces() if err != nil { setupLog.Error(err, "failed to get watch namespace") diff --git a/cmd/webhooks/main.go b/cmd/webhooks/main.go index 56c03eb1b00..72d452bbc1f 100644 --- a/cmd/webhooks/main.go +++ b/cmd/webhooks/main.go @@ -22,7 +22,6 @@ import ( "os" "github.com/spf13/pflag" - _ "go.uber.org/automaxprocs" apimachineryruntime "k8s.io/apimachinery/pkg/runtime" utilruntime "k8s.io/apimachinery/pkg/util/runtime" clientgoscheme "k8s.io/client-go/kubernetes/scheme" @@ -80,6 +79,12 @@ func main() { ctrl.SetLogger(zap.New(zap.UseFlagOptions(&opts))) + err := kedautil.ConfigureMaxProcs(setupLog) + if err != nil { + setupLog.Error(err, "failed to set max procs") + os.Exit(1) + } + ctx := ctrl.SetupSignalHandler() cfg := ctrl.GetConfigOrDie() diff --git a/pkg/util/maxprocs.go b/pkg/util/maxprocs.go new file mode 100644 index 00000000000..2cedc59185a --- /dev/null +++ b/pkg/util/maxprocs.go @@ -0,0 +1,34 @@ +/* +Copyright 2024 The KEDA 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 util + +import ( + "fmt" + + "go.uber.org/automaxprocs/maxprocs" + "k8s.io/klog/v2" +) + +// ConfigureMaxProcs sets up automaxprocs with proper logging configuration. +// It wraps the automaxprocs logger to handle structured logging with string keys +// to prevent panics when automaxprocs tries to pass numeric keys. +func ConfigureMaxProcs(logger klog.Logger) error { + _, err := maxprocs.Set(maxprocs.Logger(func(format string, args ...interface{}) { + logger.Info(fmt.Sprintf(format, args...)) + })) + return err +} From 92f353317fa17e8e679bc53d6ccade0cdd14169d Mon Sep 17 00:00:00 2001 From: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> Date: Sat, 23 Nov 2024 12:35:36 -0800 Subject: [PATCH 03/15] feat: add nsq scaler (#6230) * feat: add nsq scaler Signed-off-by: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> * fix changelog and image ref Signed-off-by: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> * fix changelog formatting Signed-off-by: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> * fix: address comments Signed-off-by: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> * feat: add useHttps/unsafeSsl to nsqScaler Signed-off-by: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> * fix: make nsq e2e tests more reliable Signed-off-by: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> * Update tests/scalers/nsq/nsq_test.go Co-authored-by: Jorge Turrado Ferrero Signed-off-by: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> * fix: update changelog Signed-off-by: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> --------- Signed-off-by: Matt Ulmer <25484774+Ulminator@users.noreply.github.com> Co-authored-by: Jorge Turrado Ferrero --- CHANGELOG.md | 2 +- pkg/scalers/nsq_scaler.go | 369 ++++++++++++++++++ pkg/scalers/nsq_scaler_test.go | 664 +++++++++++++++++++++++++++++++++ pkg/scaling/scalers_builder.go | 2 + tests/scalers/nsq/nsq_test.go | 224 +++++++++++ 5 files changed, 1260 insertions(+), 1 deletion(-) create mode 100644 pkg/scalers/nsq_scaler.go create mode 100644 pkg/scalers/nsq_scaler_test.go create mode 100644 tests/scalers/nsq/nsq_test.go diff --git a/CHANGELOG.md b/CHANGELOG.md index df2a1987770..be694eb88b1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -58,7 +58,7 @@ To learn more about active deprecations, we recommend checking [GitHub Discussio ### New -- TODO ([#XXX](https://github.com/kedacore/keda/issues/XXX)) +- **General**: Introduce new NSQ scaler ([#3281](https://github.com/kedacore/keda/issues/3281)) #### Experimental diff --git a/pkg/scalers/nsq_scaler.go b/pkg/scalers/nsq_scaler.go new file mode 100644 index 00000000000..651f1947d92 --- /dev/null +++ b/pkg/scalers/nsq_scaler.go @@ -0,0 +1,369 @@ +package scalers + +import ( + "context" + "encoding/json" + "fmt" + "io" + "net" + "net/http" + "net/url" + "strconv" + "sync" + + "github.com/go-logr/logr" + v2 "k8s.io/api/autoscaling/v2" + "k8s.io/metrics/pkg/apis/external_metrics" + + "github.com/kedacore/keda/v2/pkg/scalers/scalersconfig" + kedautil "github.com/kedacore/keda/v2/pkg/util" +) + +type nsqScaler struct { + metricType v2.MetricTargetType + metadata nsqMetadata + httpClient *http.Client + scheme string + logger logr.Logger +} + +type nsqMetadata struct { + NSQLookupdHTTPAddresses []string `keda:"name=nsqLookupdHTTPAddresses, order=triggerMetadata;resolvedEnv"` + Topic string `keda:"name=topic, order=triggerMetadata;resolvedEnv"` + Channel string `keda:"name=channel, order=triggerMetadata;resolvedEnv"` + DepthThreshold int64 `keda:"name=depthThreshold, order=triggerMetadata;resolvedEnv, default=10"` + ActivationDepthThreshold int64 `keda:"name=activationDepthThreshold, order=triggerMetadata;resolvedEnv, default=0"` + UseHTTPS bool `keda:"name=useHttps, order=triggerMetadata;resolvedEnv, default=false"` + UnsafeSSL bool `keda:"name=unsafeSsl, order=triggerMetadata;resolvedEnv, default=false"` + + triggerIndex int +} + +const ( + nsqMetricType = "External" +) + +func NewNSQScaler(config *scalersconfig.ScalerConfig) (Scaler, error) { + metricType, err := GetMetricTargetType(config) + if err != nil { + return nil, fmt.Errorf("error getting scaler metric type: %w", err) + } + + logger := InitializeLogger(config, "nsq_scaler") + + nsqMetadata, err := parseNSQMetadata(config) + if err != nil { + return nil, fmt.Errorf("error parsing NSQ metadata: %w", err) + } + + scheme := "http" + if nsqMetadata.UseHTTPS { + scheme = "https" + } + + return &nsqScaler{ + metricType: metricType, + metadata: nsqMetadata, + httpClient: kedautil.CreateHTTPClient(config.GlobalHTTPTimeout, nsqMetadata.UnsafeSSL), + scheme: scheme, + logger: logger, + }, nil +} + +func (m nsqMetadata) Validate() error { + if len(m.NSQLookupdHTTPAddresses) == 0 { + return fmt.Errorf("no nsqLookupdHTTPAddresses given") + } + + if m.DepthThreshold <= 0 { + return fmt.Errorf("depthThreshold must be a positive integer") + } + + if m.ActivationDepthThreshold < 0 { + return fmt.Errorf("activationDepthThreshold must be greater than or equal to 0") + } + + return nil +} + +func parseNSQMetadata(config *scalersconfig.ScalerConfig) (nsqMetadata, error) { + meta := nsqMetadata{triggerIndex: config.TriggerIndex} + if err := config.TypedConfig(&meta); err != nil { + return meta, fmt.Errorf("error parsing nsq metadata: %w", err) + } + + return meta, nil +} + +func (s nsqScaler) GetMetricsAndActivity(ctx context.Context, metricName string) ([]external_metrics.ExternalMetricValue, bool, error) { + depth, err := s.getTopicChannelDepth(ctx) + + if err != nil { + return []external_metrics.ExternalMetricValue{}, false, err + } + + s.logger.V(1).Info("GetMetricsAndActivity", "metricName", metricName, "depth", depth) + + metric := GenerateMetricInMili(metricName, float64(depth)) + + return []external_metrics.ExternalMetricValue{metric}, depth > s.metadata.ActivationDepthThreshold, nil +} + +func (s nsqScaler) getTopicChannelDepth(ctx context.Context) (int64, error) { + nsqdHosts, err := s.getTopicProducers(ctx, s.metadata.Topic) + if err != nil { + return -1, fmt.Errorf("error getting nsqd hosts: %w", err) + } + + if len(nsqdHosts) == 0 { + s.logger.V(1).Info("no nsqd hosts found for topic", "topic", s.metadata.Topic) + return 0, nil + } + + depth, err := s.aggregateDepth(ctx, nsqdHosts, s.metadata.Topic, s.metadata.Channel) + if err != nil { + return -1, fmt.Errorf("error getting topic/channel depth: %w", err) + } + + return depth, nil +} + +func (s nsqScaler) GetMetricSpecForScaling(context.Context) []v2.MetricSpec { + metricName := fmt.Sprintf("nsq-%s-%s", s.metadata.Topic, s.metadata.Channel) + + externalMetric := &v2.ExternalMetricSource{ + Metric: v2.MetricIdentifier{ + Name: GenerateMetricNameWithIndex(s.metadata.triggerIndex, kedautil.NormalizeString(metricName)), + }, + Target: GetMetricTarget(s.metricType, s.metadata.DepthThreshold), + } + metricSpec := v2.MetricSpec{External: externalMetric, Type: nsqMetricType} + return []v2.MetricSpec{metricSpec} +} + +func (s nsqScaler) Close(context.Context) error { + if s.httpClient != nil { + s.httpClient.CloseIdleConnections() + } + return nil +} + +type lookupResponse struct { + Producers []struct { + HTTPPort int `json:"http_port"` + BroadcastAddress string `json:"broadcast_address"` + } +} + +type lookupResult struct { + host string + lookupResponse *lookupResponse + err error +} + +func (s *nsqScaler) getTopicProducers(ctx context.Context, topic string) ([]string, error) { + var wg sync.WaitGroup + resultCh := make(chan lookupResult, len(s.metadata.NSQLookupdHTTPAddresses)) + + for _, host := range s.metadata.NSQLookupdHTTPAddresses { + wg.Add(1) + go func(host string, topic string) { + defer wg.Done() + resp, err := s.getLookup(ctx, host, topic) + resultCh <- lookupResult{host, resp, err} + }(host, topic) + } + + wg.Wait() + close(resultCh) + + var nsqdHostMap = make(map[string]bool) + for result := range resultCh { + if result.err != nil { + return nil, fmt.Errorf("error getting lookup from host '%s': %w", result.host, result.err) + } + + if result.lookupResponse == nil { + // topic is not found on a single nsqlookupd host, it may exist on another + continue + } + + for _, producer := range result.lookupResponse.Producers { + nsqdHost := net.JoinHostPort(producer.BroadcastAddress, strconv.Itoa(producer.HTTPPort)) + nsqdHostMap[nsqdHost] = true + } + } + + var nsqdHosts []string + for nsqdHost := range nsqdHostMap { + nsqdHosts = append(nsqdHosts, nsqdHost) + } + + return nsqdHosts, nil +} + +func (s *nsqScaler) getLookup(ctx context.Context, host string, topic string) (*lookupResponse, error) { + lookupURL := url.URL{ + Scheme: s.scheme, + Host: host, + Path: "lookup", + } + req, err := http.NewRequestWithContext(ctx, "GET", lookupURL.String(), nil) + if err != nil { + return nil, err + } + req.Header.Set("Accept", "application/json; charset=utf-8") + + params := url.Values{"topic": {topic}} + req.URL.RawQuery = params.Encode() + + resp, err := s.httpClient.Do(req) + if err != nil { + return nil, err + } + defer resp.Body.Close() + + if resp.StatusCode == http.StatusNotFound { + return nil, nil + } + + if resp.StatusCode != http.StatusOK { + return nil, fmt.Errorf("unexpected status code '%s'", resp.Status) + } + + body, err := io.ReadAll(resp.Body) + if err != nil { + return nil, err + } + + var lookupResponse lookupResponse + err = json.Unmarshal(body, &lookupResponse) + if err != nil { + return nil, err + } + + return &lookupResponse, nil +} + +type statsResponse struct { + Topics []struct { + TopicName string `json:"topic_name"` + Depth int64 `json:"depth"` + Channels []struct { + ChannelName string `json:"channel_name"` + Depth int64 `json:"depth"` // num messages in the queue (mem + disk) + Paused bool `json:"paused"` // if paused, consumers will not receive messages + } + } +} + +type statsResult struct { + host string + statsResponse *statsResponse + err error +} + +func (s *nsqScaler) aggregateDepth(ctx context.Context, nsqdHosts []string, topic string, channel string) (int64, error) { + wg := sync.WaitGroup{} + resultCh := make(chan statsResult, len(nsqdHosts)) + + for _, host := range nsqdHosts { + wg.Add(1) + go func(host string, topic string) { + defer wg.Done() + resp, err := s.getStats(ctx, host, topic) + resultCh <- statsResult{host, resp, err} + }(host, topic) + } + + wg.Wait() + close(resultCh) + + var depth int64 + for result := range resultCh { + if result.err != nil { + return -1, fmt.Errorf("error getting stats from host '%s': %w", result.host, result.err) + } + + for _, t := range result.statsResponse.Topics { + if t.TopicName != topic { + // this should never happen as we make the /stats call with the "topic" param + continue + } + + if len(t.Channels) == 0 { + // topic exists with no channels, but there are messages in the topic -> we should still scale to bootstrap + s.logger.V(1).Info("no channels exist for topic", "topic", topic, "channel", channel, "host", result.host) + depth += t.Depth + continue + } + + channelExists := false + for _, ch := range t.Channels { + if ch.ChannelName != channel { + continue + } + channelExists = true + if ch.Paused { + // if it's paused on a single nsqd host, it's depth should not go into the aggregate + // meaning if paused on all nsqd hosts => depth == 0 + s.logger.V(1).Info("channel is paused", "topic", topic, "channel", channel, "host", result.host) + continue + } + depth += ch.Depth + } + if !channelExists { + // topic exists with channels, but not the one in question - fallback to topic depth + s.logger.V(1).Info("channel does not exist for topic", "topic", topic, "channel", channel, "host", result.host) + depth += t.Depth + } + } + } + + return depth, nil +} + +func (s *nsqScaler) getStats(ctx context.Context, host string, topic string) (*statsResponse, error) { + statsURL := url.URL{ + Scheme: s.scheme, + Host: host, + Path: "stats", + } + req, err := http.NewRequestWithContext(ctx, "GET", statsURL.String(), nil) + if err != nil { + return nil, err + } + + // "channel" is a query param as well, but if used and the channel does not exist + // we do not receive any stats for the existing topic + params := url.Values{ + "format": {"json"}, + "include_clients": {"false"}, + "include_mem": {"false"}, + "topic": {topic}, + } + req.URL.RawQuery = params.Encode() + + resp, err := s.httpClient.Do(req) + if err != nil { + return nil, err + } + defer resp.Body.Close() + + if resp.StatusCode != http.StatusOK { + return nil, fmt.Errorf("unexpected status code '%s'", resp.Status) + } + + body, err := io.ReadAll(resp.Body) + if err != nil { + return nil, err + } + + var statsResponse statsResponse + err = json.Unmarshal(body, &statsResponse) + if err != nil { + return nil, err + } + + return &statsResponse, nil +} diff --git a/pkg/scalers/nsq_scaler_test.go b/pkg/scalers/nsq_scaler_test.go new file mode 100644 index 00000000000..956f9f1621e --- /dev/null +++ b/pkg/scalers/nsq_scaler_test.go @@ -0,0 +1,664 @@ +package scalers + +import ( + "context" + "fmt" + "net" + "net/http" + "net/http/httptest" + "net/url" + "testing" + + "github.com/go-logr/logr" + "github.com/stretchr/testify/assert" + "go.uber.org/atomic" + v2 "k8s.io/api/autoscaling/v2" + + "github.com/kedacore/keda/v2/pkg/scalers/scalersconfig" +) + +type nsqMetadataTestData struct { + metadata map[string]string + numNSQLookupdHTTPAddresses int + nsqLookupdHTTPAddresses []string + topic string + channel string + depthThreshold int64 + activationDepthThreshold int64 + useHTTPS bool + unsafeSsl bool + isError bool + description string +} + +type nsqMetricIdentifier struct { + metadataTestData *nsqMetadataTestData + triggerIndex int + name string + metricType string +} + +var parseNSQMetadataTestDataset = []nsqMetadataTestData{ + { + metadata: map[string]string{"nsqLookupdHTTPAddresses": "nsqlookupd-0:4161", "topic": "topic", "channel": "channel"}, + numNSQLookupdHTTPAddresses: 1, + nsqLookupdHTTPAddresses: []string{"nsqlookupd-0:4161"}, + topic: "topic", + channel: "channel", + depthThreshold: 10, + activationDepthThreshold: 0, + isError: false, + description: "Success", + }, + { + metadata: map[string]string{"nsqLookupdHTTPAddresses": "nsqlookupd-0:4161,nsqlookupd-1:4161", "topic": "topic", "channel": "channel"}, + numNSQLookupdHTTPAddresses: 2, + nsqLookupdHTTPAddresses: []string{"nsqlookupd-0:4161", "nsqlookupd-1:4161"}, + topic: "topic", + channel: "channel", + depthThreshold: 10, + activationDepthThreshold: 0, + isError: false, + description: "Success, multiple nsqlookupd addresses", + }, + { + metadata: map[string]string{"nsqLookupdHTTPAddresses": "nsqlookupd-0:4161", "topic": "topic", "channel": "channel", "depthThreshold": "100", "activationDepthThreshold": "1", "useHttps": "true", "unsafeSsl": "true"}, + numNSQLookupdHTTPAddresses: 1, + nsqLookupdHTTPAddresses: []string{"nsqlookupd-0:4161"}, + topic: "topic", + channel: "channel", + depthThreshold: 100, + activationDepthThreshold: 1, + useHTTPS: true, + unsafeSsl: true, + isError: false, + description: "Success - setting optional fields", + }, + { + metadata: map[string]string{"topic": "topic", "channel": "channel"}, + isError: true, + description: "Error, no nsqlookupd addresses", + }, + { + metadata: map[string]string{"nsqLookupdHTTPAddresses": "nsqlookupd-0:4161", "channel": "channel"}, + isError: true, + description: "Error, no topic", + }, + { + metadata: map[string]string{"nsqLookupdHTTPAddresses": "nsqlookupd-0:4161", "topic": "topic"}, + isError: true, + description: "Error, no channel", + }, + { + metadata: map[string]string{"nsqLookupdHTTPAddresses": "nsqlookupd-0:4161", "topic": "topic", "channel": "channel", "depthThreshold": "0"}, + isError: true, + description: "Error, depthThreshold is <=0", + }, + { + metadata: map[string]string{"nsqLookupdHTTPAddresses": "nsqlookupd-0:4161", "topic": "topic", "channel": "channel", "activationDepthThreshold": "-1"}, + isError: true, + description: "Error, activationDepthThreshold is <0", + }, +} + +var nsqMetricIdentifiers = []nsqMetricIdentifier{ + {&parseNSQMetadataTestDataset[0], 0, "s0-nsq-topic-channel", "Value"}, + {&parseNSQMetadataTestDataset[0], 1, "s1-nsq-topic-channel", "AverageValue"}, +} + +func TestNSQParseMetadata(t *testing.T) { + for _, testData := range parseNSQMetadataTestDataset { + config := scalersconfig.ScalerConfig{TriggerMetadata: testData.metadata} + + meta, err := parseNSQMetadata(&config) + if err != nil { + if testData.isError { + continue + } + t.Error("Expected success, got error", err, testData.description) + } + if err == nil && testData.isError { + t.Error("Expected error, got success", testData.description) + } + + assert.Equal(t, testData.numNSQLookupdHTTPAddresses, len(meta.NSQLookupdHTTPAddresses), testData.description) + assert.Equal(t, testData.nsqLookupdHTTPAddresses, meta.NSQLookupdHTTPAddresses, testData.description) + assert.Equal(t, testData.topic, meta.Topic, testData.description) + assert.Equal(t, testData.channel, meta.Channel, testData.description) + assert.Equal(t, testData.depthThreshold, meta.DepthThreshold, testData.description) + assert.Equal(t, testData.activationDepthThreshold, meta.ActivationDepthThreshold, testData.description) + assert.Equal(t, testData.useHTTPS, meta.UseHTTPS, testData.description) + assert.Equal(t, testData.unsafeSsl, meta.UnsafeSSL, testData.description) + } +} + +func TestNSQGetMetricsAndActivity(t *testing.T) { + type testCase struct { + lookupError bool + statsError bool + expectedDepth int64 + expectedActive bool + activationdDepthThreshold int64 + } + testCases := []testCase{ + { + lookupError: true, + }, + { + statsError: true, + }, + { + expectedDepth: 100, + expectedActive: true, + }, + { + expectedDepth: 0, + expectedActive: false, + }, + { + expectedDepth: 9, + activationdDepthThreshold: 10, + expectedActive: false, + }, + } + for _, tc := range testCases { + mockNSQdServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(http.StatusOK) + // nosemgrep: no-fprintf-to-responsewriter + fmt.Fprintf(w, `{"topics":[{"topic_name":"topic","channels":[{"channel_name":"channel","depth":%d}]}]}`, tc.expectedDepth) + })) + defer mockNSQdServer.Close() + + parsedNSQdURL, err := url.Parse(mockNSQdServer.URL) + assert.Nil(t, err) + + mockNSQLookupdServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(http.StatusOK) + // nosemgrep: no-fprintf-to-responsewriter + fmt.Fprintf(w, `{"producers":[{"broadcast_address":"%s","http_port":%s}]}`, parsedNSQdURL.Hostname(), parsedNSQdURL.Port()) + })) + defer mockNSQLookupdServer.Close() + + parsedNSQLookupdURL, err := url.Parse(mockNSQLookupdServer.URL) + assert.Nil(t, err) + + nsqlookupdHost := net.JoinHostPort(parsedNSQLookupdURL.Hostname(), parsedNSQLookupdURL.Port()) + + config := scalersconfig.ScalerConfig{TriggerMetadata: map[string]string{ + "nsqLookupdHTTPAddresses": nsqlookupdHost, + "topic": "topic", + "channel": "channel", + "activationDepthThreshold": fmt.Sprintf("%d", tc.activationdDepthThreshold), + }} + meta, err := parseNSQMetadata(&config) + assert.Nil(t, err) + + s := nsqScaler{v2.AverageValueMetricType, meta, http.DefaultClient, "http", logr.Discard()} + + metricName := "s0-nsq-topic-channel" + metrics, activity, err := s.GetMetricsAndActivity(context.Background(), metricName) + + if err != nil && (tc.lookupError || tc.statsError) { + assert.Equal(t, 0, len(metrics)) + assert.False(t, activity) + continue + } + + assert.Nil(t, err) + assert.Equal(t, 1, len(metrics)) + assert.Equal(t, metricName, metrics[0].MetricName) + assert.Equal(t, tc.expectedDepth, metrics[0].Value.Value()) + if tc.expectedActive { + assert.True(t, activity) + } else { + assert.False(t, activity) + } + } +} + +func TestNSQGetMetricSpecForScaling(t *testing.T) { + for _, testData := range nsqMetricIdentifiers { + meta, err := parseNSQMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadataTestData.metadata, TriggerIndex: testData.triggerIndex}) + if err != nil { + t.Fatal("Could not parse metadata:", err) + } + + metricType := v2.MetricTargetType(testData.metricType) + mockNSQScaler := nsqScaler{metricType, meta, nil, "http", logr.Discard()} + + metricSpec := mockNSQScaler.GetMetricSpecForScaling(context.Background()) + metricName := metricSpec[0].External.Metric.Name + assert.Equal(t, testData.name, metricName) + assert.Equal(t, 1, len(metricSpec)) + assert.Equal(t, metricType, metricSpec[0].External.Target.Type) + depthThreshold := meta.DepthThreshold + if metricType == v2.AverageValueMetricType { + assert.Equal(t, depthThreshold, metricSpec[0].External.Target.AverageValue.Value()) + } else { + assert.Equal(t, depthThreshold, metricSpec[0].External.Target.Value.Value()) + } + } +} + +func TestNSQGetTopicChannelDepth(t *testing.T) { + type testCase struct { + lookupError bool + topicNotExist bool + producersNotExist bool + statsError bool + channelPaused bool + expectedDepth int64 + description string + } + testCases := []testCase{ + { + lookupError: true, + description: "nsqlookupd call failed", + }, + { + topicNotExist: true, + expectedDepth: 0, + description: "Topic does not exist", + }, + { + producersNotExist: true, + expectedDepth: 0, + description: "No producers for topic", + }, + { + statsError: true, + description: "nsqd call failed", + }, + { + channelPaused: true, + expectedDepth: 0, + description: "Channel is paused", + }, + { + expectedDepth: 100, + description: "successfully retrieved depth", + }, + } + + for _, tc := range testCases { + mockNSQdServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if tc.statsError { + w.WriteHeader(http.StatusInternalServerError) + return + } + if tc.channelPaused { + w.WriteHeader(http.StatusOK) + fmt.Fprint(w, `{"topics":[{"topic_name":"topic", "depth":250, "channels":[{"channel_name":"channel", "depth":100, "paused":true}]}]}`) + return + } + + w.WriteHeader(http.StatusOK) + fmt.Fprint(w, `{"topics":[{"topic_name":"topic", "depth":250, "channels":[{"channel_name":"channel", "depth":100}]}]}`) + })) + defer mockNSQdServer.Close() + + parsedNSQdURL, err := url.Parse(mockNSQdServer.URL) + assert.Nil(t, err) + + mockNSQLookupdServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if tc.lookupError { + w.WriteHeader(http.StatusInternalServerError) + return + } + if tc.topicNotExist { + w.WriteHeader(http.StatusOK) + fmt.Fprint(w, `{"message": "TOPIC_NOT_FOUND"}`) + return + } + if tc.producersNotExist { + w.WriteHeader(http.StatusOK) + fmt.Fprint(w, `{"producers":[]}`) + return + } + + w.WriteHeader(http.StatusOK) + // nosemgrep: no-fprintf-to-responsewriter + fmt.Fprintf(w, `{"producers":[{"broadcast_address":"%s","http_port":%s}]}`, parsedNSQdURL.Hostname(), parsedNSQdURL.Port()) + })) + defer mockNSQLookupdServer.Close() + + parsedNSQLookupdURL, err := url.Parse(mockNSQLookupdServer.URL) + assert.Nil(t, err) + + nsqLookupdHosts := []string{net.JoinHostPort(parsedNSQLookupdURL.Hostname(), parsedNSQLookupdURL.Port())} + + s := nsqScaler{httpClient: http.DefaultClient, scheme: "http", metadata: nsqMetadata{NSQLookupdHTTPAddresses: nsqLookupdHosts, Topic: "topic", Channel: "channel"}} + + depth, err := s.getTopicChannelDepth(context.Background()) + + if err != nil && (tc.lookupError || tc.statsError) { + continue + } + + assert.Nil(t, err) + assert.Equal(t, tc.expectedDepth, depth) + } +} + +func TestNSQGetTopicProducers(t *testing.T) { + type statusAndResponse struct { + status int + response string + } + type testCase struct { + statusAndResponses []statusAndResponse + expectedNSQdHosts []string + isError bool + description string + } + testCases := []testCase{ + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"producers":[], "channels":[]}`}, + }, + expectedNSQdHosts: []string{}, + description: "No producers or channels", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"producers":[{"broadcast_address":"nsqd-0","http_port":4161}]}`}, + }, + expectedNSQdHosts: []string{"nsqd-0:4161"}, + description: "Single nsqd host", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"producers":[{"broadcast_address":"nsqd-0","http_port":4161}, {"broadcast_address":"nsqd-1","http_port":4161}]}`}, + {http.StatusOK, `{"producers":[{"broadcast_address":"nsqd-2","http_port":8161}]}`}, + }, + expectedNSQdHosts: []string{"nsqd-0:4161", "nsqd-1:4161", "nsqd-2:8161"}, + description: "Multiple nsqd hosts", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"producers":[{"broadcast_address":"nsqd-0","http_port":4161}]}`}, + {http.StatusOK, `{"producers":[{"broadcast_address":"nsqd-0","http_port":4161}]}`}, + }, + expectedNSQdHosts: []string{"nsqd-0:4161"}, + description: "De-dupe nsqd hosts", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"producers":[{"broadcast_address":"nsqd-0","http_port":4161}]}`}, + {http.StatusInternalServerError, ""}, + }, + isError: true, + description: "At least one host responded with error", + }, + } + + for _, tc := range testCases { + callCount := atomic.NewInt32(-1) + mockServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + callCount.Inc() + w.WriteHeader(tc.statusAndResponses[callCount.Load()].status) + // nosemgrep: no-fprintf-to-responsewriter + fmt.Fprint(w, tc.statusAndResponses[callCount.Load()].response) + })) + defer mockServer.Close() + + parsedURL, err := url.Parse(mockServer.URL) + assert.Nil(t, err) + + var nsqLookupdHosts []string + nsqLookupdHost := net.JoinHostPort(parsedURL.Hostname(), parsedURL.Port()) + for i := 0; i < len(tc.statusAndResponses); i++ { + nsqLookupdHosts = append(nsqLookupdHosts, nsqLookupdHost) + } + + s := nsqScaler{httpClient: http.DefaultClient, scheme: "http", metadata: nsqMetadata{NSQLookupdHTTPAddresses: nsqLookupdHosts}} + + nsqdHosts, err := s.getTopicProducers(context.Background(), "topic") + + if err != nil && tc.isError { + continue + } + + assert.Nil(t, err) + assert.ElementsMatch(t, tc.expectedNSQdHosts, nsqdHosts) + } +} + +func TestNSQGetLookup(t *testing.T) { + type testCase struct { + serverStatus int + serverResponse string + isError bool + description string + } + testCases := []testCase{ + { + serverStatus: http.StatusNotFound, + serverResponse: `{"message": "TOPIC_NOT_FOUND"}`, + isError: false, + description: "Topic does not exist", + }, + { + serverStatus: http.StatusOK, + serverResponse: `{"producers":[{"broadcast_address":"nsqd-0","http_port":4151}], "channels":[]}`, + isError: false, + description: "Channel does not exist", + }, + { + serverStatus: http.StatusNotFound, + serverResponse: `{"producers":[], "channels":["channel"]}`, + isError: false, + description: "No nsqd producers exist", + }, + { + serverStatus: http.StatusOK, + serverResponse: `{"producers":[{"broadcast_address":"nsqd-0", "http_port":4151}], "channels":["channel"]}`, + isError: false, + description: "Topic and channel exist with nsqd producers", + }, + { + serverStatus: http.StatusInternalServerError, + isError: true, + description: "Host responds with error", + }, + } + + s := nsqScaler{httpClient: http.DefaultClient, scheme: "http"} + for _, tc := range testCases { + mockServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(tc.serverStatus) + // nosemgrep: no-fprintf-to-responsewriter + fmt.Fprint(w, tc.serverResponse) + })) + defer mockServer.Close() + + parsedURL, err := url.Parse(mockServer.URL) + assert.Nil(t, err) + + host := net.JoinHostPort(parsedURL.Hostname(), parsedURL.Port()) + + resp, err := s.getLookup(context.Background(), host, "topic") + + if err != nil && tc.isError { + continue + } + + assert.Nil(t, err, tc.description) + + if tc.serverStatus != http.StatusNotFound { + assert.NotNil(t, resp, tc.description) + } else { + assert.Nil(t, resp, tc.description) + } + } +} + +func TestNSQAggregateDepth(t *testing.T) { + type statusAndResponse struct { + status int + response string + } + type testCase struct { + statusAndResponses []statusAndResponse + expectedDepth int64 + isError bool + description string + } + testCases := []testCase{ + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"topics":null}`}, + }, + expectedDepth: 0, + isError: false, + description: "Topic does not exist", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":250, "channels":[]}]}`}, + }, + expectedDepth: 250, + isError: false, + description: "Topic exists with no channels", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":250, "channels":[{"channel_name":"other_channel", "depth":100}]}]}`}, + }, + expectedDepth: 250, + isError: false, + description: "Topic exists with different channels", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":250, "channels":[{"channel_name":"channel", "depth":100}]}]}`}, + }, + expectedDepth: 100, + isError: false, + description: "Topic and channel exist", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":250, "channels":[{"channel_name":"channel", "depth":100, "paused":true}]}]}`}, + }, + expectedDepth: 0, + isError: false, + description: "Channel is paused", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":250, "channels":[{"channel_name":"channel", "depth":100}]}]}`}, + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":250, "channels":[{"channel_name":"channel", "depth":50}]}]}`}, + }, + expectedDepth: 150, + isError: false, + description: "Sum multiple depth values", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":500, "channels":[]}]}`}, + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":400, "channels":[{"channel_name":"other_channel", "depth":300}]}]}`}, + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":200, "channels":[{"channel_name":"channel", "depth":100}]}]}`}, + }, + expectedDepth: 1000, + isError: false, + description: "Channel doesn't exist on all nsqd hosts", + }, + { + statusAndResponses: []statusAndResponse{ + {http.StatusOK, `{"topics":[{"topic_name":"topic", "depth":250, "channels":[{"channel_name":"channel", "depth":100}]}]}`}, + {http.StatusInternalServerError, ""}, + }, + expectedDepth: -1, + isError: true, + description: "At least one host responded with error", + }, + } + + s := nsqScaler{httpClient: http.DefaultClient, scheme: "http"} + for _, tc := range testCases { + callCount := atomic.NewInt32(-1) + mockServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + callCount.Inc() + w.WriteHeader(tc.statusAndResponses[callCount.Load()].status) + // nosemgrep: no-fprintf-to-responsewriter + fmt.Fprint(w, tc.statusAndResponses[callCount.Load()].response) + })) + defer mockServer.Close() + + parsedURL, err := url.Parse(mockServer.URL) + assert.Nil(t, err) + + var nsqdHosts []string + nsqdHost := net.JoinHostPort(parsedURL.Hostname(), parsedURL.Port()) + for i := 0; i < len(tc.statusAndResponses); i++ { + nsqdHosts = append(nsqdHosts, nsqdHost) + } + + depth, err := s.aggregateDepth(context.Background(), nsqdHosts, "topic", "channel") + + if err != nil && tc.isError { + continue + } + + assert.Nil(t, err, tc.description) + assert.Equal(t, tc.expectedDepth, depth, tc.description) + } +} + +func TestNSQGetStats(t *testing.T) { + type testCase struct { + serverStatus int + serverResponse string + isError bool + description string + } + testCases := []testCase{ + { + serverStatus: http.StatusOK, + serverResponse: `{"topics":null}`, + isError: false, + description: "Topic does not exist", + }, + { + serverStatus: http.StatusOK, + serverResponse: `{"topics":[{"topic_name":"topic", "depth":250, "channels":[]}]}`, + isError: false, + description: "Channel does not exist", + }, + { + serverStatus: http.StatusOK, + serverResponse: `{"topics":[{"topic_name":"topic", "depth":250, "channels":[{"channel_name":"channel", "depth":250}]}]}`, + isError: false, + description: "Topic and channel exist", + }, + { + serverStatus: http.StatusInternalServerError, + isError: true, + description: "Host responds with error", + }, + } + + s := nsqScaler{httpClient: http.DefaultClient, scheme: "http"} + for _, tc := range testCases { + mockServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(tc.serverStatus) + // nosemgrep: no-fprintf-to-responsewriter + fmt.Fprint(w, tc.serverResponse) + })) + defer mockServer.Close() + + parsedURL, err := url.Parse(mockServer.URL) + assert.Nil(t, err) + + host := net.JoinHostPort(parsedURL.Hostname(), parsedURL.Port()) + resp, err := s.getStats(context.Background(), host, "topic") + + if err != nil && tc.isError { + continue + } + + assert.Nil(t, err, tc.description) + assert.NotNil(t, resp, tc.description) + } +} diff --git a/pkg/scaling/scalers_builder.go b/pkg/scaling/scalers_builder.go index 80bfb40658f..9c44ac5004e 100644 --- a/pkg/scaling/scalers_builder.go +++ b/pkg/scaling/scalers_builder.go @@ -217,6 +217,8 @@ func buildScaler(ctx context.Context, client client.Client, triggerType string, return scalers.NewNATSJetStreamScaler(config) case "new-relic": return scalers.NewNewRelicScaler(config) + case "nsq": + return scalers.NewNSQScaler(config) case "openstack-metric": return scalers.NewOpenstackMetricScaler(ctx, config) case "openstack-swift": diff --git a/tests/scalers/nsq/nsq_test.go b/tests/scalers/nsq/nsq_test.go new file mode 100644 index 00000000000..cb81c506bdd --- /dev/null +++ b/tests/scalers/nsq/nsq_test.go @@ -0,0 +1,224 @@ +//go:build e2e +// +build e2e + +package nsq_test + +import ( + "fmt" + "testing" + + "github.com/joho/godotenv" + "github.com/stretchr/testify/require" + "k8s.io/client-go/kubernetes" + + . "github.com/kedacore/keda/v2/tests/helper" +) + +var _ = godotenv.Load("../../.env") + +const ( + testName = "nsq-test" +) + +var ( + testNamespace = fmt.Sprintf("%s-ns", testName) + deploymentName = fmt.Sprintf("%s-consumer-deployment", testName) + jobName = fmt.Sprintf("%s-producer-job", testName) + scaledObjectName = fmt.Sprintf("%s-so", testName) + nsqNamespace = "nsq" + nsqHelmRepoURL = "https://nsqio.github.io/helm-chart" + minReplicaCount = 0 + maxReplicaCount = 2 + topicName = "test_topic" + channelName = "test_channel" + depthThreshold = 1 + activationDepthThreshold = 5 +) + +const ( + deploymentTemplate = ` +apiVersion: apps/v1 +kind: Deployment +metadata: + name: {{.DeploymentName}} + namespace: {{.TestNamespace}} + labels: + app: {{.DeploymentName}} +spec: + selector: + matchLabels: + app: {{.DeploymentName}} + template: + metadata: + labels: + app: {{.DeploymentName}} + spec: + containers: + - image: ghcr.io/kedacore/tests-nsq:latest + name: {{.DeploymentName}} + args: + - "--mode=consumer" + - "--topic={{.TopicName}}" + - "--channel={{.ChannelName}}" + - "--sleep-duration=1s" + - "--nsqlookupd-http-address=nsq-nsqlookupd.{{.NSQNamespace}}.svc.cluster.local:4161" + imagePullPolicy: Always +` + + scaledObjectTemplate = ` +apiVersion: keda.sh/v1alpha1 +kind: ScaledObject +metadata: + name: {{.ScaledObjectName}} + namespace: {{.TestNamespace}} + labels: + app: {{.DeploymentName}} +spec: + pollingInterval: 5 + cooldownPeriod: 10 + maxReplicaCount: {{.MaxReplicaCount}} + minReplicaCount: {{.MinReplicaCount}} + scaleTargetRef: + apiVersion: "apps/v1" + kind: "Deployment" + name: {{.DeploymentName}} + triggers: + - type: nsq + metricType: "AverageValue" + metadata: + nsqLookupdHTTPAddresses: "nsq-nsqlookupd.{{.NSQNamespace}}.svc.cluster.local:4161" + topic: "{{.TopicName}}" + channel: "{{.ChannelName}}" + depthThreshold: "{{.DepthThreshold}}" + activationDepthThreshold: "{{.ActivationDepthThreshold}}" +` + + jobTemplate = ` +apiVersion: batch/v1 +kind: Job +metadata: + name: {{.JobName}} + namespace: {{.TestNamespace}} +spec: + template: + spec: + containers: + - image: ghcr.io/kedacore/tests-nsq:latest + name: {{.JobName}} + args: + - "--mode=producer" + - "--topic={{.TopicName}}" + - "--nsqd-tcp-address=nsq-nsqd.{{.NSQNamespace}}.svc.cluster.local:4150" + - "--message-count={{.MessageCount}}" + imagePullPolicy: Always + restartPolicy: Never +` +) + +type templateData struct { + TestNamespace string + NSQNamespace string + DeploymentName string + ScaledObjectName string + JobName string + MinReplicaCount int + MaxReplicaCount int + TopicName string + ChannelName string + DepthThreshold int + ActivationDepthThreshold int + MessageCount int +} + +func TestNSQScaler(t *testing.T) { + kc := GetKubernetesClient(t) + + t.Cleanup(func() { + data, templates := getTemplateData() + uninstallNSQ(t) + KubectlDeleteWithTemplate(t, data, "jobTemplate", jobTemplate) + DeleteKubernetesResources(t, testNamespace, data, templates) + }) + + installNSQ(t, kc) + + data, templates := getTemplateData() + CreateKubernetesResources(t, kc, testNamespace, data, templates) + + require.True(t, WaitForPodsTerminated(t, kc, fmt.Sprintf("app=%s", deploymentName), testNamespace, 60, 1), + "Replica count should start out as 0") + + testActivation(t, kc, data) + testScaleOut(t, kc, data) + testScaleIn(t, kc) +} + +func installNSQ(t *testing.T, kc *kubernetes.Clientset) { + t.Log("--- installing NSQ ---") + CreateNamespace(t, kc, nsqNamespace) + + _, err := ExecuteCommand("which helm") + require.NoErrorf(t, err, "nsq test requires helm - %s", err) + + _, err = ExecuteCommand(fmt.Sprintf("helm repo add nsqio %s", nsqHelmRepoURL)) + require.NoErrorf(t, err, "error while adding nsqio helm repo - %s", err) + + _, err = ExecuteCommand(fmt.Sprintf("helm install nsq nsqio/nsq --namespace %s --set nsqd.replicaCount=1 --set nsqlookupd.replicaCount=1 --set nsqadmin.enabled=false --wait", nsqNamespace)) + require.NoErrorf(t, err, "error while installing nsq - %s", err) +} + +func uninstallNSQ(t *testing.T) { + t.Log("--- uninstalling NSQ ---") + _, err := ExecuteCommand(fmt.Sprintf("helm uninstall nsq --namespace %s", nsqNamespace)) + require.NoErrorf(t, err, "error while uninstalling nsq - %s", err) + DeleteNamespace(t, nsqNamespace) +} + +func getTemplateData() (templateData, []Template) { + return templateData{ + TestNamespace: testNamespace, + NSQNamespace: nsqNamespace, + DeploymentName: deploymentName, + JobName: jobName, + ScaledObjectName: scaledObjectName, + MinReplicaCount: minReplicaCount, + MaxReplicaCount: maxReplicaCount, + TopicName: topicName, + ChannelName: channelName, + DepthThreshold: depthThreshold, + ActivationDepthThreshold: activationDepthThreshold, + }, []Template{ + {Name: "deploymentTemplate", Config: deploymentTemplate}, + {Name: "scaledObjectTemplate", Config: scaledObjectTemplate}, + } +} + +func testActivation(t *testing.T, kc *kubernetes.Clientset, data templateData) { + t.Log("--- testing activation ---") + + data.MessageCount = activationDepthThreshold + KubectlReplaceWithTemplate(t, data, "jobTemplate", jobTemplate) + AssertReplicaCountNotChangeDuringTimePeriod(t, kc, deploymentName, testNamespace, 0, 20) + + data.MessageCount = 1 + KubectlReplaceWithTemplate(t, data, "jobTemplate", jobTemplate) + require.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 1, 60, 1), + "replica count should reach 1 in under 1 minute") +} + +func testScaleOut(t *testing.T, kc *kubernetes.Clientset, data templateData) { + t.Log("--- testing scale out ---") + + data.MessageCount = 80 + KubectlReplaceWithTemplate(t, data, "jobTemplate", jobTemplate) + + require.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, maxReplicaCount, 60, 1), + "replica count should reach 2 in under 1 minute") +} + +func testScaleIn(t *testing.T, kc *kubernetes.Clientset) { + t.Log("--- testing scale in ---") + + require.True(t, WaitForDeploymentReplicaReadyCount(t, kc, deploymentName, testNamespace, 0, 60, 1), + "replica count should be 0 after 1 minute") +} From 9831734ac48793637dacbc6bf2b35f1ffd69f9e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A5rten=20Svantesson?= Date: Sun, 24 Nov 2024 00:25:25 +0200 Subject: [PATCH 04/15] fix: scaledjobs stuck as not ready (#6329) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit since code was missing for setting a scaledjob as ready it was stuck as unready if there ever was a problem This is a fix for a regression in #5916 Signed-off-by: Mårten Svantesson --- CHANGELOG.md | 1 + pkg/scaling/executor/scale_jobs.go | 11 ++++++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index be694eb88b1..7da6fd76866 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -74,6 +74,7 @@ Here is an overview of all new **experimental** features: - **General**: Centralize and improve automaxprocs configuration with proper structured logging ([#5970](https://github.com/kedacore/keda/issues/5970)) - **General**: Paused ScaledObject count is reported correctly after operator restart ([#6321](https://github.com/kedacore/keda/issues/6321)) +- **General**: ScaledJobs ready status set to true when recoverred problem ([#6329](https://github.com/kedacore/keda/pull/6329)) ### Deprecations diff --git a/pkg/scaling/executor/scale_jobs.go b/pkg/scaling/executor/scale_jobs.go index b32eb344f07..c1dc4adf315 100644 --- a/pkg/scaling/executor/scale_jobs.go +++ b/pkg/scaling/executor/scale_jobs.go @@ -65,10 +65,10 @@ func (e *scaleExecutor) RequestJobScale(ctx context.Context, scaledJob *kedav1al logger.V(1).Info("No change in activity") } + readyCondition := scaledJob.Status.Conditions.GetReadyCondition() if isError { // some triggers responded with error // Set ScaledJob.Status.ReadyCondition to Unknown - readyCondition := scaledJob.Status.Conditions.GetReadyCondition() msg := "Some triggers defined in ScaledJob are not working correctly" logger.V(1).Info(msg) if !readyCondition.IsUnknown() { @@ -76,6 +76,15 @@ func (e *scaleExecutor) RequestJobScale(ctx context.Context, scaledJob *kedav1al logger.Error(err, "error setting ready condition") } } + } else if !readyCondition.IsTrue() { + // if the ScaledObject's triggers aren't in the error state, + // but ScaledJob.Status.ReadyCondition is set not set to 'true' -> set it back to 'true' + msg := "ScaledJob is defined correctly and is ready for scaling" + logger.V(1).Info(msg) + if err := e.setReadyCondition(ctx, logger, scaledJob, metav1.ConditionTrue, + "ScaledJobReady", msg); err != nil { + logger.Error(err, "error setting ready condition") + } } condition := scaledJob.Status.Conditions.GetActiveCondition() From 29400ed2816fe7388a021ff14f5cb405b7deaa85 Mon Sep 17 00:00:00 2001 From: rickbrouwer Date: Tue, 26 Nov 2024 19:29:11 +0100 Subject: [PATCH 05/15] Deprecate Stan scaler (#6363) Signed-off-by: rickbrouwer --- CHANGELOG.md | 2 +- pkg/scalers/stan_scaler.go | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7da6fd76866..8660a1b2123 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -82,7 +82,7 @@ You can find all deprecations in [this overview](https://github.com/kedacore/ked New deprecation(s): -- TODO ([#XXX](https://github.com/kedacore/keda/issues/XXX)) +- **NATS Streaming scaler**: Deprecate NATS Streaming Server (aka Stan) ([#6362](https://github.com/kedacore/keda/issues/6362)) ### Breaking Changes diff --git a/pkg/scalers/stan_scaler.go b/pkg/scalers/stan_scaler.go index 701073e7d12..b4930d7bc31 100644 --- a/pkg/scalers/stan_scaler.go +++ b/pkg/scalers/stan_scaler.go @@ -278,6 +278,7 @@ func (s *stanScaler) GetMetricsAndActivity(ctx context.Context, metricName strin } totalLag := s.getMaxMsgLag() s.logger.V(1).Info("Stan scaler: Providing metrics based on totalLag, threshold", "totalLag", totalLag, "lagThreshold", s.metadata.lagThreshold) + s.logger.Info("The Stan scaler (NATS Streaming) is DEPRECATED and will be removed in v2.19 - Use scaler 'nats-jetstream' instead") metric := GenerateMetricInMili(metricName, float64(totalLag)) From 5c58849f2eb38409d60c418e81664c970127dbb5 Mon Sep 17 00:00:00 2001 From: Omer Aplatony Date: Wed, 27 Nov 2024 23:36:44 +0200 Subject: [PATCH 06/15] refactor aws sqs queue scaler configuration (#6358) Signed-off-by: Omer Aplatony --- pkg/scalers/aws_sqs_queue_scaler.go | 115 +++++------------------ pkg/scalers/aws_sqs_queue_scaler_test.go | 22 ++--- 2 files changed, 34 insertions(+), 103 deletions(-) diff --git a/pkg/scalers/aws_sqs_queue_scaler.go b/pkg/scalers/aws_sqs_queue_scaler.go index 1de9bf7f285..6f1b6b5d0ee 100644 --- a/pkg/scalers/aws_sqs_queue_scaler.go +++ b/pkg/scalers/aws_sqs_queue_scaler.go @@ -19,14 +19,6 @@ import ( kedautil "github.com/kedacore/keda/v2/pkg/util" ) -const ( - defaultTargetQueueLength = 5 - targetQueueLengthDefault = 5 - activationTargetQueueLengthDefault = 0 - defaultScaleOnInFlight = true - defaultScaleOnDelayed = false -) - type awsSqsQueueScaler struct { metricType v2.MetricTargetType metadata *awsSqsQueueMetadata @@ -35,16 +27,16 @@ type awsSqsQueueScaler struct { } type awsSqsQueueMetadata struct { - targetQueueLength int64 - activationTargetQueueLength int64 - queueURL string + TargetQueueLength int64 `keda:"name=queueLength, order=triggerMetadata, default=5"` + ActivationTargetQueueLength int64 `keda:"name=activationQueueLength, order=triggerMetadata, default=0"` + QueueURL string `keda:"name=queueURL;queueURLFromEnv, order=triggerMetadata;resolvedEnv"` queueName string - awsRegion string - awsEndpoint string + AwsRegion string `keda:"name=awsRegion, order=triggerMetadata"` + AwsEndpoint string `keda:"name=awsEndpoint, order=triggerMetadata, optional"` awsAuthorization awsutils.AuthorizationMetadata triggerIndex int - scaleOnInFlight bool - scaleOnDelayed bool + ScaleOnInFlight bool `keda:"name=scaleOnInFlight, order=triggerMetadata, default=true"` + ScaleOnDelayed bool `keda:"name=scaleOnDelayed, order=triggerMetadata, default=false"` awsSqsQueueMetricNames []types.QueueAttributeName } @@ -57,7 +49,7 @@ func NewAwsSqsQueueScaler(ctx context.Context, config *scalersconfig.ScalerConfi logger := InitializeLogger(config, "aws_sqs_queue_scaler") - meta, err := parseAwsSqsQueueMetadata(config, logger) + meta, err := parseAwsSqsQueueMetadata(config) if err != nil { return nil, fmt.Errorf("error parsing SQS queue metadata: %w", err) } @@ -87,77 +79,26 @@ func (w sqsWrapperClient) GetQueueAttributes(ctx context.Context, params *sqs.Ge return w.sqsClient.GetQueueAttributes(ctx, params, optFns...) } -func parseAwsSqsQueueMetadata(config *scalersconfig.ScalerConfig, logger logr.Logger) (*awsSqsQueueMetadata, error) { - meta := awsSqsQueueMetadata{} - meta.targetQueueLength = defaultTargetQueueLength - meta.scaleOnInFlight = defaultScaleOnInFlight - meta.scaleOnDelayed = defaultScaleOnDelayed - - if val, ok := config.TriggerMetadata["queueLength"]; ok && val != "" { - queueLength, err := strconv.ParseInt(val, 10, 64) - if err != nil { - meta.targetQueueLength = targetQueueLengthDefault - logger.Error(err, "Error parsing SQS queue metadata queueLength, using default %n", targetQueueLengthDefault) - } else { - meta.targetQueueLength = queueLength - } - } - - if val, ok := config.TriggerMetadata["activationQueueLength"]; ok && val != "" { - activationQueueLength, err := strconv.ParseInt(val, 10, 64) - if err != nil { - meta.activationTargetQueueLength = activationTargetQueueLengthDefault - logger.Error(err, "Error parsing SQS queue metadata activationQueueLength, using default %n", activationTargetQueueLengthDefault) - } else { - meta.activationTargetQueueLength = activationQueueLength - } - } - - if val, ok := config.TriggerMetadata["scaleOnDelayed"]; ok && val != "" { - scaleOnDelayed, err := strconv.ParseBool(val) - if err != nil { - meta.scaleOnDelayed = defaultScaleOnDelayed - logger.Error(err, "Error parsing SQS queue metadata scaleOnDelayed, using default %n", defaultScaleOnDelayed) - } else { - meta.scaleOnDelayed = scaleOnDelayed - } - } +func parseAwsSqsQueueMetadata(config *scalersconfig.ScalerConfig) (*awsSqsQueueMetadata, error) { + meta := &awsSqsQueueMetadata{} - if val, ok := config.TriggerMetadata["scaleOnInFlight"]; ok && val != "" { - scaleOnInFlight, err := strconv.ParseBool(val) - if err != nil { - meta.scaleOnInFlight = defaultScaleOnInFlight - logger.Error(err, "Error parsing SQS queue metadata scaleOnInFlight, using default %n", defaultScaleOnInFlight) - } else { - meta.scaleOnInFlight = scaleOnInFlight - } + if err := config.TypedConfig(meta); err != nil { + return nil, fmt.Errorf("error parsing SQS queue metadata: %w", err) } meta.awsSqsQueueMetricNames = []types.QueueAttributeName{} meta.awsSqsQueueMetricNames = append(meta.awsSqsQueueMetricNames, types.QueueAttributeNameApproximateNumberOfMessages) - if meta.scaleOnInFlight { + if meta.ScaleOnInFlight { meta.awsSqsQueueMetricNames = append(meta.awsSqsQueueMetricNames, types.QueueAttributeNameApproximateNumberOfMessagesNotVisible) } - if meta.scaleOnDelayed { + if meta.ScaleOnDelayed { meta.awsSqsQueueMetricNames = append(meta.awsSqsQueueMetricNames, types.QueueAttributeNameApproximateNumberOfMessagesDelayed) } - if val, ok := config.TriggerMetadata["queueURL"]; ok && val != "" { - meta.queueURL = val - } else if val, ok := config.TriggerMetadata["queueURLFromEnv"]; ok && val != "" { - if val, ok := config.ResolvedEnv[val]; ok && val != "" { - meta.queueURL = val - } else { - return nil, fmt.Errorf("queueURLFromEnv `%s` env variable value is empty", config.TriggerMetadata["queueURLFromEnv"]) - } - } else { - return nil, fmt.Errorf("no queueURL given") - } - - queueURL, err := url.ParseRequestURI(meta.queueURL) + queueURL, err := url.ParseRequestURI(meta.QueueURL) if err != nil { // queueURL is not a valid URL, using it as queueName - meta.queueName = meta.queueURL + meta.queueName = meta.QueueURL } else { queueURLPath := queueURL.Path queueURLPathParts := strings.Split(queueURLPath, "/") @@ -168,16 +109,6 @@ func parseAwsSqsQueueMetadata(config *scalersconfig.ScalerConfig, logger logr.Lo meta.queueName = queueURLPathParts[2] } - if val, ok := config.TriggerMetadata["awsRegion"]; ok && val != "" { - meta.awsRegion = val - } else { - return nil, fmt.Errorf("no awsRegion given") - } - - if val, ok := config.TriggerMetadata["awsEndpoint"]; ok { - meta.awsEndpoint = val - } - auth, err := awsutils.GetAwsAuthorization(config.TriggerUniqueKey, config.PodIdentity, config.TriggerMetadata, config.AuthParams, config.ResolvedEnv) if err != nil { return nil, err @@ -187,17 +118,17 @@ func parseAwsSqsQueueMetadata(config *scalersconfig.ScalerConfig, logger logr.Lo meta.triggerIndex = config.TriggerIndex - return &meta, nil + return meta, nil } func createSqsClient(ctx context.Context, metadata *awsSqsQueueMetadata) (*sqs.Client, error) { - cfg, err := awsutils.GetAwsConfig(ctx, metadata.awsRegion, metadata.awsAuthorization) + cfg, err := awsutils.GetAwsConfig(ctx, metadata.AwsRegion, metadata.awsAuthorization) if err != nil { return nil, err } return sqs.NewFromConfig(*cfg, func(options *sqs.Options) { - if metadata.awsEndpoint != "" { - options.BaseEndpoint = aws.String(metadata.awsEndpoint) + if metadata.AwsEndpoint != "" { + options.BaseEndpoint = aws.String(metadata.AwsEndpoint) } }), nil } @@ -212,7 +143,7 @@ func (s *awsSqsQueueScaler) GetMetricSpecForScaling(context.Context) []v2.Metric Metric: v2.MetricIdentifier{ Name: GenerateMetricNameWithIndex(s.metadata.triggerIndex, kedautil.NormalizeString(fmt.Sprintf("aws-sqs-%s", s.metadata.queueName))), }, - Target: GetMetricTarget(s.metricType, s.metadata.targetQueueLength), + Target: GetMetricTarget(s.metricType, s.metadata.TargetQueueLength), } metricSpec := v2.MetricSpec{External: externalMetric, Type: externalMetricType} return []v2.MetricSpec{metricSpec} @@ -229,14 +160,14 @@ func (s *awsSqsQueueScaler) GetMetricsAndActivity(ctx context.Context, metricNam metric := GenerateMetricInMili(metricName, float64(queuelen)) - return []external_metrics.ExternalMetricValue{metric}, queuelen > s.metadata.activationTargetQueueLength, nil + return []external_metrics.ExternalMetricValue{metric}, queuelen > s.metadata.ActivationTargetQueueLength, nil } // Get SQS Queue Length func (s *awsSqsQueueScaler) getAwsSqsQueueLength(ctx context.Context) (int64, error) { input := &sqs.GetQueueAttributesInput{ AttributeNames: s.metadata.awsSqsQueueMetricNames, - QueueUrl: aws.String(s.metadata.queueURL), + QueueUrl: aws.String(s.metadata.QueueURL), } output, err := s.sqsWrapperClient.GetQueueAttributes(ctx, input) diff --git a/pkg/scalers/aws_sqs_queue_scaler_test.go b/pkg/scalers/aws_sqs_queue_scaler_test.go index 6e0b065ab07..fb5db401ecc 100644 --- a/pkg/scalers/aws_sqs_queue_scaler_test.go +++ b/pkg/scalers/aws_sqs_queue_scaler_test.go @@ -144,8 +144,8 @@ var testAWSSQSMetadata = []parseAWSSQSMetadataTestData{ "awsRegion": "eu-west-1"}, testAWSSQSAuthentication, testAWSSQSEmptyResolvedEnv, - false, - "properly formed queue, invalid queueLength"}, + true, + "invalid integer value for queueLength"}, {map[string]string{ "queueURL": testAWSSQSProperQueueURL, "queueLength": "1", @@ -162,8 +162,8 @@ var testAWSSQSMetadata = []parseAWSSQSMetadataTestData{ "awsRegion": "eu-west-1"}, testAWSSQSAuthentication, testAWSSQSEmptyResolvedEnv, - false, - "properly formed queue, invalid activationQueueLength"}, + true, + "invalid integer value for activationQueueLength"}, {map[string]string{ "queueURL": testAWSSQSProperQueueURL, "queueLength": "1", @@ -304,7 +304,7 @@ var testAWSSQSMetadata = []parseAWSSQSMetadataTestData{ map[string]string{ "QUEUE_URL": "", }, - true, + false, "empty QUEUE_URL env value"}, } @@ -392,7 +392,7 @@ var awsSQSGetMetricTestData = []*parseAWSSQSMetadataTestData{ func TestSQSParseMetadata(t *testing.T) { for _, testData := range testAWSSQSMetadata { - _, err := parseAwsSqsQueueMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadata, ResolvedEnv: testData.resolvedEnv, AuthParams: testData.authParams}, logr.Discard()) + _, err := parseAwsSqsQueueMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadata, ResolvedEnv: testData.resolvedEnv, AuthParams: testData.authParams}) if err != nil && !testData.isError { t.Errorf("Expected success because %s got error, %s", testData.comment, err) } @@ -405,7 +405,7 @@ func TestSQSParseMetadata(t *testing.T) { func TestAWSSQSGetMetricSpecForScaling(t *testing.T) { for _, testData := range awsSQSMetricIdentifiers { ctx := context.Background() - meta, err := parseAwsSqsQueueMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadataTestData.metadata, ResolvedEnv: testData.metadataTestData.resolvedEnv, AuthParams: testData.metadataTestData.authParams, TriggerIndex: testData.triggerIndex}, logr.Discard()) + meta, err := parseAwsSqsQueueMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadataTestData.metadata, ResolvedEnv: testData.metadataTestData.resolvedEnv, AuthParams: testData.metadataTestData.authParams, TriggerIndex: testData.triggerIndex}) if err != nil { t.Fatal("Could not parse metadata:", err) } @@ -421,24 +421,24 @@ func TestAWSSQSGetMetricSpecForScaling(t *testing.T) { func TestAWSSQSScalerGetMetrics(t *testing.T) { for index, testData := range awsSQSGetMetricTestData { - meta, err := parseAwsSqsQueueMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadata, ResolvedEnv: testData.resolvedEnv, AuthParams: testData.authParams, TriggerIndex: index}, logr.Discard()) + meta, err := parseAwsSqsQueueMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadata, ResolvedEnv: testData.resolvedEnv, AuthParams: testData.authParams, TriggerIndex: index}) if err != nil { t.Fatal("Could not parse metadata:", err) } scaler := awsSqsQueueScaler{"", meta, &mockSqs{}, logr.Discard()} value, _, err := scaler.GetMetricsAndActivity(context.Background(), "MetricName") - switch meta.queueURL { + switch meta.QueueURL { case testAWSSQSErrorQueueURL: assert.Error(t, err, "expect error because of sqs api error") case testAWSSQSBadDataQueueURL: assert.Error(t, err, "expect error because of bad data return from sqs") default: expectedMessages := testAWSSQSApproximateNumberOfMessagesVisible - if meta.scaleOnInFlight { + if meta.ScaleOnInFlight { expectedMessages += testAWSSQSApproximateNumberOfMessagesNotVisible } - if meta.scaleOnDelayed { + if meta.ScaleOnDelayed { expectedMessages += testAWSSQSApproximateNumberOfMessagesDelayed } assert.EqualValues(t, int64(expectedMessages), value[0].Value.Value()) From 2adbc81571e5301f69fa3a591f5e397f0b564aa1 Mon Sep 17 00:00:00 2001 From: Omer Aplatony Date: Thu, 28 Nov 2024 00:03:21 +0200 Subject: [PATCH 07/15] Refactor AWS DynamoDB Streams scaler configuration (#6351) * Refactor AWS DynamoDB Streams scaler configuration Signed-off-by: Omer Aplatony * fixed unit tests Signed-off-by: Omer Aplatony * Fix invalid value test Signed-off-by: Omer Aplatony * go fmt Signed-off-by: Omer Aplatony --------- Signed-off-by: Omer Aplatony --- pkg/scalers/aws_dynamodb_streams_scaler.go | 69 ++++----------- .../aws_dynamodb_streams_scaler_test.go | 84 ++++++++----------- 2 files changed, 55 insertions(+), 98 deletions(-) diff --git a/pkg/scalers/aws_dynamodb_streams_scaler.go b/pkg/scalers/aws_dynamodb_streams_scaler.go index a8448a46408..cdcd8548320 100644 --- a/pkg/scalers/aws_dynamodb_streams_scaler.go +++ b/pkg/scalers/aws_dynamodb_streams_scaler.go @@ -3,7 +3,6 @@ package scalers import ( "context" "fmt" - "strconv" "github.com/aws/aws-sdk-go-v2/aws" "github.com/aws/aws-sdk-go-v2/service/dynamodb" @@ -31,11 +30,11 @@ type awsDynamoDBStreamsScaler struct { } type awsDynamoDBStreamsMetadata struct { - targetShardCount int64 - activationTargetShardCount int64 - tableName string - awsRegion string - awsEndpoint string + TargetShardCount int64 `keda:"name=shardCount, order=triggerMetadata, default=2"` + ActivationTargetShardCount int64 `keda:"name=activationShardCount, order=triggerMetadata, default=0"` + TableName string `keda:"name=tableName, order=triggerMetadata"` + AwsRegion string `keda:"name=awsRegion, order=triggerMetadata"` + AwsEndpoint string `keda:"name=awsEndpoint, order=triggerMetadata, optional"` awsAuthorization awsutils.AuthorizationMetadata triggerIndex int } @@ -49,7 +48,7 @@ func NewAwsDynamoDBStreamsScaler(ctx context.Context, config *scalersconfig.Scal logger := InitializeLogger(config, "aws_dynamodb_streams_scaler") - meta, err := parseAwsDynamoDBStreamsMetadata(config, logger) + meta, err := parseAwsDynamoDBStreamsMetadata(config) if err != nil { return nil, fmt.Errorf("error parsing dynamodb stream metadata: %w", err) } @@ -58,7 +57,7 @@ func NewAwsDynamoDBStreamsScaler(ctx context.Context, config *scalersconfig.Scal if err != nil { return nil, fmt.Errorf("error when creating dynamodbstream client: %w", err) } - streamArn, err := getDynamoDBStreamsArn(ctx, dbClient, &meta.tableName) + streamArn, err := getDynamoDBStreamsArn(ctx, dbClient, &meta.TableName) if err != nil { return nil, fmt.Errorf("error dynamodb stream arn: %w", err) } @@ -74,43 +73,11 @@ func NewAwsDynamoDBStreamsScaler(ctx context.Context, config *scalersconfig.Scal }, nil } -func parseAwsDynamoDBStreamsMetadata(config *scalersconfig.ScalerConfig, logger logr.Logger) (*awsDynamoDBStreamsMetadata, error) { +func parseAwsDynamoDBStreamsMetadata(config *scalersconfig.ScalerConfig) (*awsDynamoDBStreamsMetadata, error) { meta := awsDynamoDBStreamsMetadata{} - meta.targetShardCount = defaultTargetDBStreamsShardCount - if val, ok := config.TriggerMetadata["awsRegion"]; ok && val != "" { - meta.awsRegion = val - } else { - return nil, fmt.Errorf("no awsRegion given") - } - - if val, ok := config.TriggerMetadata["awsEndpoint"]; ok { - meta.awsEndpoint = val - } - - if val, ok := config.TriggerMetadata["tableName"]; ok && val != "" { - meta.tableName = val - } else { - return nil, fmt.Errorf("no tableName given") - } - - if val, ok := config.TriggerMetadata["shardCount"]; ok && val != "" { - shardCount, err := strconv.ParseInt(val, 10, 64) - if err != nil { - meta.targetShardCount = defaultTargetDBStreamsShardCount - logger.Error(err, "error parsing dyanmodb stream metadata shardCount, using default %n", defaultTargetDBStreamsShardCount) - } else { - meta.targetShardCount = shardCount - } - } - if val, ok := config.TriggerMetadata["activationShardCount"]; ok && val != "" { - shardCount, err := strconv.ParseInt(val, 10, 64) - if err != nil { - meta.activationTargetShardCount = defaultActivationTargetDBStreamsShardCount - logger.Error(err, "error parsing dyanmodb stream metadata activationTargetShardCount, using default %n", defaultActivationTargetDBStreamsShardCount) - } else { - meta.activationTargetShardCount = shardCount - } + if err := config.TypedConfig(&meta); err != nil { + return nil, fmt.Errorf("error parsing dynamodb stream metadata: %w", err) } auth, err := awsutils.GetAwsAuthorization(config.TriggerUniqueKey, config.PodIdentity, config.TriggerMetadata, config.AuthParams, config.ResolvedEnv) @@ -125,18 +92,18 @@ func parseAwsDynamoDBStreamsMetadata(config *scalersconfig.ScalerConfig, logger } func createClientsForDynamoDBStreamsScaler(ctx context.Context, metadata *awsDynamoDBStreamsMetadata) (*dynamodb.Client, *dynamodbstreams.Client, error) { - cfg, err := awsutils.GetAwsConfig(ctx, metadata.awsRegion, metadata.awsAuthorization) + cfg, err := awsutils.GetAwsConfig(ctx, metadata.AwsRegion, metadata.awsAuthorization) if err != nil { return nil, nil, err } dbClient := dynamodb.NewFromConfig(*cfg, func(options *dynamodb.Options) { - if metadata.awsEndpoint != "" { - options.BaseEndpoint = aws.String(metadata.awsEndpoint) + if metadata.AwsEndpoint != "" { + options.BaseEndpoint = aws.String(metadata.AwsEndpoint) } }) dbStreamClient := dynamodbstreams.NewFromConfig(*cfg, func(options *dynamodbstreams.Options) { - if metadata.awsEndpoint != "" { - options.BaseEndpoint = aws.String(metadata.awsEndpoint) + if metadata.AwsEndpoint != "" { + options.BaseEndpoint = aws.String(metadata.AwsEndpoint) } }) @@ -176,9 +143,9 @@ func (s *awsDynamoDBStreamsScaler) Close(_ context.Context) error { func (s *awsDynamoDBStreamsScaler) GetMetricSpecForScaling(_ context.Context) []v2.MetricSpec { externalMetric := &v2.ExternalMetricSource{ Metric: v2.MetricIdentifier{ - Name: GenerateMetricNameWithIndex(s.metadata.triggerIndex, kedautil.NormalizeString(fmt.Sprintf("aws-dynamodb-streams-%s", s.metadata.tableName))), + Name: GenerateMetricNameWithIndex(s.metadata.triggerIndex, kedautil.NormalizeString(fmt.Sprintf("aws-dynamodb-streams-%s", s.metadata.TableName))), }, - Target: GetMetricTarget(s.metricType, s.metadata.targetShardCount), + Target: GetMetricTarget(s.metricType, s.metadata.TargetShardCount), } metricSpec := v2.MetricSpec{External: externalMetric, Type: externalMetricType} return []v2.MetricSpec{metricSpec} @@ -195,7 +162,7 @@ func (s *awsDynamoDBStreamsScaler) GetMetricsAndActivity(ctx context.Context, me metric := GenerateMetricInMili(metricName, float64(shardCount)) - return []external_metrics.ExternalMetricValue{metric}, shardCount > s.metadata.activationTargetShardCount, nil + return []external_metrics.ExternalMetricValue{metric}, shardCount > s.metadata.ActivationTargetShardCount, nil } // GetDynamoDBStreamShardCount Get DynamoDB Stream Shard Count diff --git a/pkg/scalers/aws_dynamodb_streams_scaler_test.go b/pkg/scalers/aws_dynamodb_streams_scaler_test.go index 5c87de8d87a..ce1232413a6 100644 --- a/pkg/scalers/aws_dynamodb_streams_scaler_test.go +++ b/pkg/scalers/aws_dynamodb_streams_scaler_test.go @@ -135,10 +135,10 @@ var testAwsDynamoDBStreamMetadata = []parseAwsDynamoDBStreamsMetadataTestData{ "awsRegion": testAWSDynamoDBStreamsRegion}, authParams: testAWSKinesisAuthentication, expected: &awsDynamoDBStreamsMetadata{ - targetShardCount: 2, - activationTargetShardCount: 1, - tableName: testAWSDynamoDBSmallTable, - awsRegion: testAWSDynamoDBStreamsRegion, + TargetShardCount: 2, + ActivationTargetShardCount: 1, + TableName: testAWSDynamoDBSmallTable, + AwsRegion: testAWSDynamoDBStreamsRegion, awsAuthorization: awsutils.AuthorizationMetadata{ AwsAccessKeyID: testAWSDynamoDBStreamsAccessKeyID, AwsSecretAccessKey: testAWSDynamoDBStreamsSecretAccessKey, @@ -159,11 +159,11 @@ var testAwsDynamoDBStreamMetadata = []parseAwsDynamoDBStreamsMetadataTestData{ "awsEndpoint": testAWSDynamoDBStreamsEndpoint}, authParams: testAWSKinesisAuthentication, expected: &awsDynamoDBStreamsMetadata{ - targetShardCount: 2, - activationTargetShardCount: 1, - tableName: testAWSDynamoDBSmallTable, - awsRegion: testAWSDynamoDBStreamsRegion, - awsEndpoint: testAWSDynamoDBStreamsEndpoint, + TargetShardCount: 2, + ActivationTargetShardCount: 1, + TableName: testAWSDynamoDBSmallTable, + AwsRegion: testAWSDynamoDBStreamsRegion, + AwsEndpoint: testAWSDynamoDBStreamsEndpoint, awsAuthorization: awsutils.AuthorizationMetadata{ AwsAccessKeyID: testAWSDynamoDBStreamsAccessKeyID, AwsSecretAccessKey: testAWSDynamoDBStreamsSecretAccessKey, @@ -204,10 +204,10 @@ var testAwsDynamoDBStreamMetadata = []parseAwsDynamoDBStreamsMetadataTestData{ "awsRegion": testAWSDynamoDBStreamsRegion}, authParams: testAWSKinesisAuthentication, expected: &awsDynamoDBStreamsMetadata{ - targetShardCount: defaultTargetDBStreamsShardCount, - activationTargetShardCount: defaultActivationTargetDBStreamsShardCount, - tableName: testAWSDynamoDBSmallTable, - awsRegion: testAWSDynamoDBStreamsRegion, + TargetShardCount: defaultTargetDBStreamsShardCount, + ActivationTargetShardCount: defaultActivationTargetDBStreamsShardCount, + TableName: testAWSDynamoDBSmallTable, + AwsRegion: testAWSDynamoDBStreamsRegion, awsAuthorization: awsutils.AuthorizationMetadata{ AwsAccessKeyID: testAWSDynamoDBStreamsAccessKeyID, AwsSecretAccessKey: testAWSDynamoDBStreamsSecretAccessKey, @@ -224,20 +224,10 @@ var testAwsDynamoDBStreamMetadata = []parseAwsDynamoDBStreamsMetadataTestData{ "tableName": testAWSDynamoDBSmallTable, "shardCount": "a", "awsRegion": testAWSDynamoDBStreamsRegion}, - authParams: testAWSKinesisAuthentication, - expected: &awsDynamoDBStreamsMetadata{ - targetShardCount: defaultTargetDBStreamsShardCount, - tableName: testAWSDynamoDBSmallTable, - awsRegion: testAWSDynamoDBStreamsRegion, - awsAuthorization: awsutils.AuthorizationMetadata{ - AwsAccessKeyID: testAWSDynamoDBStreamsAccessKeyID, - AwsSecretAccessKey: testAWSDynamoDBStreamsSecretAccessKey, - PodIdentityOwner: true, - }, - triggerIndex: 4, - }, - isError: false, - comment: "properly formed table name and region, wrong shard count", + authParams: testAWSKinesisAuthentication, + expected: &awsDynamoDBStreamsMetadata{}, + isError: true, + comment: "invalid value - should cause error", triggerIndex: 4, }, { @@ -278,9 +268,9 @@ var testAwsDynamoDBStreamMetadata = []parseAwsDynamoDBStreamsMetadataTestData{ "awsSessionToken": testAWSDynamoDBStreamsSessionToken, }, expected: &awsDynamoDBStreamsMetadata{ - targetShardCount: 2, - tableName: testAWSDynamoDBSmallTable, - awsRegion: testAWSDynamoDBStreamsRegion, + TargetShardCount: 2, + TableName: testAWSDynamoDBSmallTable, + AwsRegion: testAWSDynamoDBStreamsRegion, awsAuthorization: awsutils.AuthorizationMetadata{ AwsAccessKeyID: testAWSDynamoDBStreamsAccessKeyID, AwsSecretAccessKey: testAWSDynamoDBStreamsSecretAccessKey, @@ -330,9 +320,9 @@ var testAwsDynamoDBStreamMetadata = []parseAwsDynamoDBStreamsMetadataTestData{ "awsRoleArn": testAWSDynamoDBStreamsRoleArn, }, expected: &awsDynamoDBStreamsMetadata{ - targetShardCount: 2, - tableName: testAWSDynamoDBSmallTable, - awsRegion: testAWSDynamoDBStreamsRegion, + TargetShardCount: 2, + TableName: testAWSDynamoDBSmallTable, + AwsRegion: testAWSDynamoDBStreamsRegion, awsAuthorization: awsutils.AuthorizationMetadata{ AwsRoleArn: testAWSDynamoDBStreamsRoleArn, PodIdentityOwner: true, @@ -350,9 +340,9 @@ var testAwsDynamoDBStreamMetadata = []parseAwsDynamoDBStreamsMetadataTestData{ "identityOwner": "operator"}, authParams: map[string]string{}, expected: &awsDynamoDBStreamsMetadata{ - targetShardCount: 2, - tableName: testAWSDynamoDBSmallTable, - awsRegion: testAWSDynamoDBStreamsRegion, + TargetShardCount: 2, + TableName: testAWSDynamoDBSmallTable, + AwsRegion: testAWSDynamoDBStreamsRegion, awsAuthorization: awsutils.AuthorizationMetadata{ PodIdentityOwner: false, }, @@ -370,15 +360,15 @@ var awsDynamoDBStreamMetricIdentifiers = []awsDynamoDBStreamsMetricIdentifier{ } var awsDynamoDBStreamsGetMetricTestData = []*awsDynamoDBStreamsMetadata{ - {tableName: testAWSDynamoDBBigTable}, - {tableName: testAWSDynamoDBSmallTable}, - {tableName: testAWSDynamoDBErrorTable}, - {tableName: testAWSDynamoDBInvalidTable}, + {TableName: testAWSDynamoDBBigTable}, + {TableName: testAWSDynamoDBSmallTable}, + {TableName: testAWSDynamoDBErrorTable}, + {TableName: testAWSDynamoDBInvalidTable}, } func TestParseAwsDynamoDBStreamsMetadata(t *testing.T) { for _, testData := range testAwsDynamoDBStreamMetadata { - result, err := parseAwsDynamoDBStreamsMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadata, ResolvedEnv: testAwsDynamoDBStreamAuthentication, AuthParams: testData.authParams, TriggerIndex: testData.triggerIndex}, logr.Discard()) + result, err := parseAwsDynamoDBStreamsMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadata, ResolvedEnv: testAwsDynamoDBStreamAuthentication, AuthParams: testData.authParams, TriggerIndex: testData.triggerIndex}) if err != nil && !testData.isError { t.Errorf("Expected success because %s got error, %s", testData.comment, err) } @@ -395,11 +385,11 @@ func TestParseAwsDynamoDBStreamsMetadata(t *testing.T) { func TestAwsDynamoDBStreamsGetMetricSpecForScaling(t *testing.T) { for _, testData := range awsDynamoDBStreamMetricIdentifiers { ctx := context.Background() - meta, err := parseAwsDynamoDBStreamsMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadataTestData.metadata, ResolvedEnv: testAwsDynamoDBStreamAuthentication, AuthParams: testData.metadataTestData.authParams, TriggerIndex: testData.triggerIndex}, logr.Discard()) + meta, err := parseAwsDynamoDBStreamsMetadata(&scalersconfig.ScalerConfig{TriggerMetadata: testData.metadataTestData.metadata, ResolvedEnv: testAwsDynamoDBStreamAuthentication, AuthParams: testData.metadataTestData.authParams, TriggerIndex: testData.triggerIndex}) if err != nil { t.Fatal("Could not parse metadata:", err) } - streamArn, err := getDynamoDBStreamsArn(ctx, &mockAwsDynamoDB{}, &meta.tableName) + streamArn, err := getDynamoDBStreamsArn(ctx, &mockAwsDynamoDB{}, &meta.TableName) if err != nil { t.Fatal("Could not get dynamodb stream arn:", err) } @@ -418,12 +408,12 @@ func TestAwsDynamoDBStreamsScalerGetMetrics(t *testing.T) { var err error var streamArn *string ctx := context.Background() - streamArn, err = getDynamoDBStreamsArn(ctx, &mockAwsDynamoDB{}, &meta.tableName) + streamArn, err = getDynamoDBStreamsArn(ctx, &mockAwsDynamoDB{}, &meta.TableName) if err == nil { scaler := awsDynamoDBStreamsScaler{"", meta, streamArn, &mockAwsDynamoDBStreams{}, logr.Discard()} value, _, err = scaler.GetMetricsAndActivity(context.Background(), "MetricName") } - switch meta.tableName { + switch meta.TableName { case testAWSDynamoDBErrorTable: assert.Error(t, err, "expect error because of dynamodb stream api error") case testAWSDynamoDBInvalidTable: @@ -442,12 +432,12 @@ func TestAwsDynamoDBStreamsScalerIsActive(t *testing.T) { var err error var streamArn *string ctx := context.Background() - streamArn, err = getDynamoDBStreamsArn(ctx, &mockAwsDynamoDB{}, &meta.tableName) + streamArn, err = getDynamoDBStreamsArn(ctx, &mockAwsDynamoDB{}, &meta.TableName) if err == nil { scaler := awsDynamoDBStreamsScaler{"", meta, streamArn, &mockAwsDynamoDBStreams{}, logr.Discard()} _, value, err = scaler.GetMetricsAndActivity(context.Background(), "MetricName") } - switch meta.tableName { + switch meta.TableName { case testAWSDynamoDBErrorTable: assert.Error(t, err, "expect error because of dynamodb stream api error") case testAWSDynamoDBInvalidTable: From 8399fae24272656b8632957f4ef30fc6da0cb106 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 10:25:10 +0100 Subject: [PATCH 08/15] chore(deps): bump aquasecurity/trivy-action from 0.28.0 to 0.29.0 (#6385) Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/template-trivy-scan.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/template-trivy-scan.yml b/.github/workflows/template-trivy-scan.yml index 385c9f0ef2e..642a59ae017 100644 --- a/.github/workflows/template-trivy-scan.yml +++ b/.github/workflows/template-trivy-scan.yml @@ -39,7 +39,7 @@ jobs: - uses: actions/checkout@11bd71901bbe5b1630ceea73d27597364c9af683 # v4 - name: Run Trivy - uses: aquasecurity/trivy-action@915b19bbe73b92a6cf82a1bc12b087c9a19a5fe2 # v0.28.0 + uses: aquasecurity/trivy-action@18f2510ee396bbf400402947b394f2dd8c87dbb0 # v0.29.0 env: TRIVY_DB_REPOSITORY: ghcr.io/kedacore/trivy-db with: From faf42ef91136efbc8402fe3357e5a68151f1ffbe Mon Sep 17 00:00:00 2001 From: Matthias Diester Date: Mon, 2 Dec 2024 15:32:23 +0100 Subject: [PATCH 09/15] Use `toolchain` to define desired Go version (#6378) --- go.mod | 6 +- go.sum | 6 +- vendor/go.uber.org/automaxprocs/.codecov.yml | 14 ---- vendor/go.uber.org/automaxprocs/.gitignore | 33 -------- vendor/go.uber.org/automaxprocs/CHANGELOG.md | 52 ------------ .../automaxprocs/CODE_OF_CONDUCT.md | 75 ----------------- .../go.uber.org/automaxprocs/CONTRIBUTING.md | 81 ------------------- vendor/go.uber.org/automaxprocs/Makefile | 46 ----------- vendor/go.uber.org/automaxprocs/README.md | 71 ---------------- .../go.uber.org/automaxprocs/automaxprocs.go | 33 -------- vendor/modules.txt | 1 - 11 files changed, 6 insertions(+), 412 deletions(-) delete mode 100644 vendor/go.uber.org/automaxprocs/.codecov.yml delete mode 100644 vendor/go.uber.org/automaxprocs/.gitignore delete mode 100644 vendor/go.uber.org/automaxprocs/CHANGELOG.md delete mode 100644 vendor/go.uber.org/automaxprocs/CODE_OF_CONDUCT.md delete mode 100644 vendor/go.uber.org/automaxprocs/CONTRIBUTING.md delete mode 100644 vendor/go.uber.org/automaxprocs/Makefile delete mode 100644 vendor/go.uber.org/automaxprocs/README.md delete mode 100644 vendor/go.uber.org/automaxprocs/automaxprocs.go diff --git a/go.mod b/go.mod index db2bf127b74..1d90fed97f9 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,8 @@ module github.com/kedacore/keda/v2 -go 1.23.3 +go 1.22.1 + +toolchain go1.23.3 require ( cloud.google.com/go/compute/metadata v0.5.2 @@ -340,7 +342,7 @@ require ( go.opentelemetry.io/otel/sdk/metric v1.31.0 go.opentelemetry.io/otel/trace v1.31.0 // indirect go.opentelemetry.io/proto/otlp v1.3.1 // indirect - go.uber.org/atomic v1.11.0 // indirect + go.uber.org/atomic v1.11.0 go.uber.org/automaxprocs v1.6.0 go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect diff --git a/go.sum b/go.sum index f1779b617be..64cfcb8f531 100644 --- a/go.sum +++ b/go.sum @@ -1623,8 +1623,8 @@ github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/elastic/go-elasticsearch/v7 v7.17.10 h1:TCQ8i4PmIJuBunvBS6bwT2ybzVFxxUhhltAs3Gyu1yo= github.com/elastic/go-elasticsearch/v7 v7.17.10/go.mod h1:OJ4wdbtDNk5g503kvlHLyErCgQwwzmDtaFC4XyOxXA4= -github.com/elazarl/goproxy v0.0.0-20220417044921-416226498f94 h1:VIy7cdK7ufs7ctpTFkXJHm1uP3dJSnCGSPysEICB1so= -github.com/elazarl/goproxy v0.0.0-20220417044921-416226498f94/go.mod h1:Ro8st/ElPeALwNFlcTpWmkr6IoMFfkjXAvTHpevnDsM= +github.com/elazarl/goproxy v0.0.0-20231117061959-7cc037d33fb5 h1:m62nsMU279qRD9PQSWD1l66kmkXzuYcnVJqL4XLeV2M= +github.com/elazarl/goproxy v0.0.0-20231117061959-7cc037d33fb5/go.mod h1:Ro8st/ElPeALwNFlcTpWmkr6IoMFfkjXAvTHpevnDsM= github.com/emicklei/go-restful/v3 v3.12.1 h1:PJMDIM/ak7btuL8Ex0iYET9hxM3CI2sjZtzpL63nKAU= github.com/emicklei/go-restful/v3 v3.12.1/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRry+4bhvbpWn3mrbc= github.com/envoyproxy/go-control-plane v0.10.2-0.20220325020618-49ff273808a1/go.mod h1:KJwIaB5Mv44NWtYuAOFCVOjcI94vtpEz2JU/D2v6IjE= @@ -2171,8 +2171,6 @@ github.com/nats-io/nkeys v0.4.7/go.mod h1:kqXRgRDPlGy7nGaEDMuYzmiJCIAAWDK0IMBtDm github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo= github.com/neelance/sourcemap v0.0.0-20200213170602-2833bce08e4c/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM= -github.com/newrelic/newrelic-client-go v1.1.0 h1:aflNjzQ21c+2GwBVh+UbAf9lznkRfCcVABoc5UM4IXw= -github.com/newrelic/newrelic-client-go v1.1.0/go.mod h1:RYMXt7hgYw7nzuXIGd2BH0F1AivgWw7WrBhNBQZEB4k= github.com/newrelic/newrelic-client-go/v2 v2.51.2 h1:Xf+M0NuZuIuxqG48zYoqyIdQL514j2J1c+kNVYajcYI= github.com/newrelic/newrelic-client-go/v2 v2.51.2/go.mod h1:+RRjI3nDGWT3kLm9Oi3QxpBm70uu8q1upEHBVWCZFpo= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= diff --git a/vendor/go.uber.org/automaxprocs/.codecov.yml b/vendor/go.uber.org/automaxprocs/.codecov.yml deleted file mode 100644 index 9a2ed4a9969..00000000000 --- a/vendor/go.uber.org/automaxprocs/.codecov.yml +++ /dev/null @@ -1,14 +0,0 @@ -coverage: - range: 80..100 - round: down - precision: 2 - - status: - project: # measuring the overall project coverage - default: # context, you can create multiple ones with custom titles - enabled: yes # must be yes|true to enable this status - target: 90% # specify the target coverage for each commit status - # option: "auto" (must increase from parent commit or pull request base) - # option: "X%" a static target percentage to hit - if_not_found: success # if parent is not found report status as success, error, or failure - if_ci_failed: error # if ci fails report status as success, error, or failure diff --git a/vendor/go.uber.org/automaxprocs/.gitignore b/vendor/go.uber.org/automaxprocs/.gitignore deleted file mode 100644 index dd7bcf5130b..00000000000 --- a/vendor/go.uber.org/automaxprocs/.gitignore +++ /dev/null @@ -1,33 +0,0 @@ -# Compiled Object files, Static and Dynamic libs (Shared Objects) -*.o -*.a -*.so - -# Folders -_obj -_test -vendor - -# Architecture specific extensions/prefixes -*.[568vq] -[568vq].out - -*.cgo1.go -*.cgo2.c -_cgo_defun.c -_cgo_gotypes.go -_cgo_export.* - -_testmain.go - -*.exe -*.test -*.prof -*.pprof -*.out -*.log -coverage.txt - -/bin -cover.out -cover.html diff --git a/vendor/go.uber.org/automaxprocs/CHANGELOG.md b/vendor/go.uber.org/automaxprocs/CHANGELOG.md deleted file mode 100644 index f421056ae82..00000000000 --- a/vendor/go.uber.org/automaxprocs/CHANGELOG.md +++ /dev/null @@ -1,52 +0,0 @@ -# Changelog - -## v1.6.0 (2024-07-24) - -- Add RoundQuotaFunc option that allows configuration of rounding - behavior for floating point CPU quota. - -## v1.5.3 (2023-07-19) - -- Fix mountinfo parsing when super options have fields with spaces. -- Fix division by zero while parsing cgroups. - -## v1.5.2 (2023-03-16) - -- Support child control cgroups -- Fix file descriptor leak -- Update dependencies - -## v1.5.1 (2022-04-06) - -- Fix cgroups v2 mountpoint detection. - -## v1.5.0 (2022-04-05) - -- Add support for cgroups v2. - -Thanks to @emadolsky for their contribution to this release. - -## v1.4.0 (2021-02-01) - -- Support colons in cgroup names. -- Remove linters from runtime dependencies. - -## v1.3.0 (2020-01-23) - -- Migrate to Go modules. - -## v1.2.0 (2018-02-22) - -- Fixed quota clamping to always round down rather than up; Rather than - guaranteeing constant throttling at saturation, instead assume that the - fractional CPU was added as a hedge for factors outside of Go's scheduler. - -## v1.1.0 (2017-11-10) - -- Log the new value of `GOMAXPROCS` rather than the current value. -- Make logs more explicit about whether `GOMAXPROCS` was modified or not. -- Allow customization of the minimum `GOMAXPROCS`, and modify default from 2 to 1. - -## v1.0.0 (2017-08-09) - -- Initial release. diff --git a/vendor/go.uber.org/automaxprocs/CODE_OF_CONDUCT.md b/vendor/go.uber.org/automaxprocs/CODE_OF_CONDUCT.md deleted file mode 100644 index e327d9aa5cd..00000000000 --- a/vendor/go.uber.org/automaxprocs/CODE_OF_CONDUCT.md +++ /dev/null @@ -1,75 +0,0 @@ -# Contributor Covenant Code of Conduct - -## Our Pledge - -In the interest of fostering an open and welcoming environment, we as -contributors and maintainers pledge to making participation in our project and -our community a harassment-free experience for everyone, regardless of age, -body size, disability, ethnicity, gender identity and expression, level of -experience, nationality, personal appearance, race, religion, or sexual -identity and orientation. - -## Our Standards - -Examples of behavior that contributes to creating a positive environment -include: - -* Using welcoming and inclusive language -* Being respectful of differing viewpoints and experiences -* Gracefully accepting constructive criticism -* Focusing on what is best for the community -* Showing empathy towards other community members - -Examples of unacceptable behavior by participants include: - -* The use of sexualized language or imagery and unwelcome sexual attention or - advances -* Trolling, insulting/derogatory comments, and personal or political attacks -* Public or private harassment -* Publishing others' private information, such as a physical or electronic - address, without explicit permission -* Other conduct which could reasonably be considered inappropriate in a - professional setting - -## Our Responsibilities - -Project maintainers are responsible for clarifying the standards of acceptable -behavior and are expected to take appropriate and fair corrective action in -response to any instances of unacceptable behavior. - -Project maintainers have the right and responsibility to remove, edit, or -reject comments, commits, code, wiki edits, issues, and other contributions -that are not aligned to this Code of Conduct, or to ban temporarily or -permanently any contributor for other behaviors that they deem inappropriate, -threatening, offensive, or harmful. - -## Scope - -This Code of Conduct applies both within project spaces and in public spaces -when an individual is representing the project or its community. Examples of -representing a project or community include using an official project e-mail -address, posting via an official social media account, or acting as an -appointed representative at an online or offline event. Representation of a -project may be further defined and clarified by project maintainers. - -## Enforcement - -Instances of abusive, harassing, or otherwise unacceptable behavior may be -reported by contacting the project team at oss-conduct@uber.com. The project -team will review and investigate all complaints, and will respond in a way -that it deems appropriate to the circumstances. The project team is obligated -to maintain confidentiality with regard to the reporter of an incident. -Further details of specific enforcement policies may be posted separately. - -Project maintainers who do not follow or enforce the Code of Conduct in good -faith may face temporary or permanent repercussions as determined by other -members of the project's leadership. - -## Attribution - -This Code of Conduct is adapted from the [Contributor Covenant][homepage], -version 1.4, available at -[http://contributor-covenant.org/version/1/4][version]. - -[homepage]: http://contributor-covenant.org -[version]: http://contributor-covenant.org/version/1/4/ diff --git a/vendor/go.uber.org/automaxprocs/CONTRIBUTING.md b/vendor/go.uber.org/automaxprocs/CONTRIBUTING.md deleted file mode 100644 index 2b6a6040d78..00000000000 --- a/vendor/go.uber.org/automaxprocs/CONTRIBUTING.md +++ /dev/null @@ -1,81 +0,0 @@ -# Contributing - -We'd love your help improving this package! - -If you'd like to add new exported APIs, please [open an issue][open-issue] -describing your proposal — discussing API changes ahead of time makes -pull request review much smoother. In your issue, pull request, and any other -communications, please remember to treat your fellow contributors with -respect! We take our [code of conduct](CODE_OF_CONDUCT.md) seriously. - -Note that you'll need to sign [Uber's Contributor License Agreement][cla] -before we can accept any of your contributions. If necessary, a bot will remind -you to accept the CLA when you open your pull request. - -## Setup - -[Fork][fork], then clone the repository: - -``` -mkdir -p $GOPATH/src/go.uber.org -cd $GOPATH/src/go.uber.org -git clone git@github.com:your_github_username/automaxprocs.git -cd automaxprocs -git remote add upstream https://github.com/uber-go/automaxprocs.git -git fetch upstream -``` - -Install the test dependencies: - -``` -make dependencies -``` - -Make sure that the tests and the linters pass: - -``` -make test -make lint -``` - -If you're not using the minor version of Go specified in the Makefile's -`LINTABLE_MINOR_VERSIONS` variable, `make lint` doesn't do anything. This is -fine, but it means that you'll only discover lint failures after you open your -pull request. - -## Making Changes - -Start by creating a new branch for your changes: - -``` -cd $GOPATH/src/go.uber.org/automaxprocs -git checkout master -git fetch upstream -git rebase upstream/master -git checkout -b cool_new_feature -``` - -Make your changes, then ensure that `make lint` and `make test` still pass. If -you're satisfied with your changes, push them to your fork. - -``` -git push origin cool_new_feature -``` - -Then use the GitHub UI to open a pull request. - -At this point, you're waiting on us to review your changes. We *try* to respond -to issues and pull requests within a few business days, and we may suggest some -improvements or alternatives. Once your changes are approved, one of the -project maintainers will merge them. - -We're much more likely to approve your changes if you: - -* Add tests for new functionality. -* Write a [good commit message][commit-message]. -* Maintain backward compatibility. - -[fork]: https://github.com/uber-go/automaxprocs/fork -[open-issue]: https://github.com/uber-go/automaxprocs/issues/new -[cla]: https://cla-assistant.io/uber-go/automaxprocs -[commit-message]: http://tbaggery.com/2008/04/19/a-note-about-git-commit-messages.html diff --git a/vendor/go.uber.org/automaxprocs/Makefile b/vendor/go.uber.org/automaxprocs/Makefile deleted file mode 100644 index 1642b714801..00000000000 --- a/vendor/go.uber.org/automaxprocs/Makefile +++ /dev/null @@ -1,46 +0,0 @@ -export GOBIN ?= $(shell pwd)/bin - -GO_FILES := $(shell \ - find . '(' -path '*/.*' -o -path './vendor' ')' -prune \ - -o -name '*.go' -print | cut -b3-) - -GOLINT = $(GOBIN)/golint -STATICCHECK = $(GOBIN)/staticcheck - -.PHONY: build -build: - go build ./... - -.PHONY: install -install: - go mod download - -.PHONY: test -test: - go test -race ./... - -.PHONY: cover -cover: - go test -coverprofile=cover.out -covermode=atomic -coverpkg=./... ./... - go tool cover -html=cover.out -o cover.html - -$(GOLINT): tools/go.mod - cd tools && go install golang.org/x/lint/golint - -$(STATICCHECK): tools/go.mod - cd tools && go install honnef.co/go/tools/cmd/staticcheck@2023.1.2 - -.PHONY: lint -lint: $(GOLINT) $(STATICCHECK) - @rm -rf lint.log - @echo "Checking gofmt" - @gofmt -d -s $(GO_FILES) 2>&1 | tee lint.log - @echo "Checking go vet" - @go vet ./... 2>&1 | tee -a lint.log - @echo "Checking golint" - @$(GOLINT) ./... | tee -a lint.log - @echo "Checking staticcheck" - @$(STATICCHECK) ./... 2>&1 | tee -a lint.log - @echo "Checking for license headers..." - @./.build/check_license.sh | tee -a lint.log - @[ ! -s lint.log ] diff --git a/vendor/go.uber.org/automaxprocs/README.md b/vendor/go.uber.org/automaxprocs/README.md deleted file mode 100644 index bfed32adae8..00000000000 --- a/vendor/go.uber.org/automaxprocs/README.md +++ /dev/null @@ -1,71 +0,0 @@ -# automaxprocs [![GoDoc][doc-img]][doc] [![Build Status][ci-img]][ci] [![Coverage Status][cov-img]][cov] - -Automatically set `GOMAXPROCS` to match Linux container CPU quota. - -## Installation - -`go get -u go.uber.org/automaxprocs` - -## Quick Start - -```go -import _ "go.uber.org/automaxprocs" - -func main() { - // Your application logic here. -} -``` - -# Performance -Data measured from Uber's internal load balancer. We ran the load balancer with 200% CPU quota (i.e., 2 cores): - -| GOMAXPROCS | RPS | P50 (ms) | P99.9 (ms) | -| ------------------ | --------- | -------- | ---------- | -| 1 | 28,893.18 | 1.46 | 19.70 | -| 2 (equal to quota) | 44,715.07 | 0.84 | 26.38 | -| 3 | 44,212.93 | 0.66 | 30.07 | -| 4 | 41,071.15 | 0.57 | 42.94 | -| 8 | 33,111.69 | 0.43 | 64.32 | -| Default (24) | 22,191.40 | 0.45 | 76.19 | - -When `GOMAXPROCS` is increased above the CPU quota, we see P50 decrease slightly, but see significant increases to P99. We also see that the total RPS handled also decreases. - -When `GOMAXPROCS` is higher than the CPU quota allocated, we also saw significant throttling: - -``` -$ cat /sys/fs/cgroup/cpu,cpuacct/system.slice/[...]/cpu.stat -nr_periods 42227334 -nr_throttled 131923 -throttled_time 88613212216618 -``` - -Once `GOMAXPROCS` was reduced to match the CPU quota, we saw no CPU throttling. - -## Development Status: Stable - -All APIs are finalized, and no breaking changes will be made in the 1.x series -of releases. Users of semver-aware dependency management systems should pin -automaxprocs to `^1`. - -## Contributing - -We encourage and support an active, healthy community of contributors — -including you! Details are in the [contribution guide](CONTRIBUTING.md) and -the [code of conduct](CODE_OF_CONDUCT.md). The automaxprocs maintainers keep -an eye on issues and pull requests, but you can also report any negative -conduct to oss-conduct@uber.com. That email list is a private, safe space; -even the automaxprocs maintainers don't have access, so don't hesitate to hold -us to a high standard. - -
- -Released under the [MIT License](LICENSE). - -[doc-img]: https://godoc.org/go.uber.org/automaxprocs?status.svg -[doc]: https://godoc.org/go.uber.org/automaxprocs -[ci-img]: https://github.com/uber-go/automaxprocs/actions/workflows/go.yml/badge.svg -[ci]: https://github.com/uber-go/automaxprocs/actions/workflows/go.yml -[cov-img]: https://codecov.io/gh/uber-go/automaxprocs/branch/master/graph/badge.svg -[cov]: https://codecov.io/gh/uber-go/automaxprocs - - diff --git a/vendor/go.uber.org/automaxprocs/automaxprocs.go b/vendor/go.uber.org/automaxprocs/automaxprocs.go deleted file mode 100644 index 69946a3e1fd..00000000000 --- a/vendor/go.uber.org/automaxprocs/automaxprocs.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright (c) 2017 Uber Technologies, Inc. -// -// 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. - -// Package automaxprocs automatically sets GOMAXPROCS to match the Linux -// container CPU quota, if any. -package automaxprocs // import "go.uber.org/automaxprocs" - -import ( - "log" - - "go.uber.org/automaxprocs/maxprocs" -) - -func init() { - maxprocs.Set(maxprocs.Logger(log.Printf)) -} diff --git a/vendor/modules.txt b/vendor/modules.txt index ba0780ef7e4..4ee7a52cdd4 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1617,7 +1617,6 @@ go.opentelemetry.io/proto/otlp/trace/v1 go.uber.org/atomic # go.uber.org/automaxprocs v1.6.0 ## explicit; go 1.20 -go.uber.org/automaxprocs go.uber.org/automaxprocs/internal/cgroups go.uber.org/automaxprocs/internal/runtime go.uber.org/automaxprocs/maxprocs From 119b0ff944c1f8a0174fa289b8e81a209aa3c0b5 Mon Sep 17 00:00:00 2001 From: rickbrouwer Date: Mon, 2 Dec 2024 19:02:57 +0100 Subject: [PATCH 10/15] Add Scorecard GitHub Action (#6365) Signed-off-by: rickbrouwer --- .github/workflows/scorecards.yml | 69 ++++++++++++++++++++++++++++++++ CHANGELOG.md | 1 + README.md | 1 + SECURITY.md | 1 + 4 files changed, 72 insertions(+) create mode 100644 .github/workflows/scorecards.yml diff --git a/.github/workflows/scorecards.yml b/.github/workflows/scorecards.yml new file mode 100644 index 00000000000..fbb28c0e0be --- /dev/null +++ b/.github/workflows/scorecards.yml @@ -0,0 +1,69 @@ +name: Scorecard supply-chain security +on: + # For Branch-Protection check. Only the default branch is supported. See + # https://github.com/ossf/scorecard/blob/main/docs/checks.md#branch-protection + branch_protection_rule: + # To guarantee Maintained check is occasionally updated. See + # https://github.com/ossf/scorecard/blob/main/docs/checks.md#maintained + schedule: + - cron: '38 12 * * 4' + push: + branches: [ "main" ] + +# Declare default permissions as read only. +permissions: read-all + +jobs: + analysis: + name: Scorecard analysis + runs-on: ubuntu-latest + permissions: + # Needed to upload the results to code-scanning dashboard. + security-events: write + # Needed to publish results and get a badge (see publish_results below). + id-token: write + # Uncomment the permissions below if installing in a private repository. + # contents: read + # actions: read + + steps: + - name: "Checkout code" + uses: actions/checkout@b4ffde65f46336ab88eb53be808477a3936bae11 # v4.1.1 + with: + persist-credentials: false + + - name: "Run analysis" + uses: ossf/scorecard-action@0864cf19026789058feabb7e87baa5f140aac736 # v2.3.1 + with: + results_file: results.sarif + results_format: sarif + # (Optional) "write" PAT token. Uncomment the `repo_token` line below if: + # - you want to enable the Branch-Protection check on a *public* repository, or + # - you are installing Scorecard on a *private* repository + # To create the PAT, follow the steps in https://github.com/ossf/scorecard-action?tab=readme-ov-file#authentication-with-fine-grained-pat-optional. + # repo_token: ${{ secrets.SCORECARD_TOKEN }} + + # Public repositories: + # - Publish results to OpenSSF REST API for easy access by consumers + # - Allows the repository to include the Scorecard badge. + # - See https://github.com/ossf/scorecard-action#publishing-results. + # For private repositories: + # - `publish_results` will always be set to `false`, regardless + # of the value entered here. + publish_results: true + + # Upload the results as artifacts (optional). Commenting out will disable uploads of run results in SARIF + # format to the repository Actions tab. + - name: "Upload artifact" + uses: actions/upload-artifact@97a0fba1372883ab732affbe8f94b823f91727db # v3.pre.node20 + with: + name: SARIF file + path: results.sarif + retention-days: 5 + + # Upload the results to GitHub's code scanning dashboard (optional). + # Commenting out will disable upload of results to your repo's Code Scanning dashboard + - name: "Upload to code-scanning" + uses: github/codeql-action/upload-sarif@1b1aada464948af03b950897e5eb522f92603cc2 # v3.24.9 + with: + sarif_file: results.sarif diff --git a/CHANGELOG.md b/CHANGELOG.md index 8660a1b2123..52538a16f71 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -58,6 +58,7 @@ To learn more about active deprecations, we recommend checking [GitHub Discussio ### New +- **General**: Enable OpenSSF Scorecard to enhance security practices across the project ([#5913](https://github.com/kedacore/keda/issues/5913)) - **General**: Introduce new NSQ scaler ([#3281](https://github.com/kedacore/keda/issues/3281)) #### Experimental diff --git a/README.md b/README.md index 375f3ed85c5..03d9b1f3d42 100644 --- a/README.md +++ b/README.md @@ -4,6 +4,7 @@ main build nightly e2e + Twitter

diff --git a/SECURITY.md b/SECURITY.md index 28cb837e400..908ee3181b9 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -15,6 +15,7 @@ We have a few preventive measures in place to detect security vulnerabilities: - All pull requests (PRs) are using CodeQL to scan our source code for vulnerabilities - Dependabot will automatically identify vulnerabilities based on GitHub Advisory Database and open PRs with patches - Automated [secret scanning](https://docs.github.com/en/enterprise-cloud@latest/code-security/secret-scanning/about-secret-scanning#about-secret-scanning-for-partner-patterns) & alerts +- The [Scorecard GitHub Action](https://github.com/ossf/scorecard-action) automates the process by running security checks on the GitHub repository. By integrating this Action into the repository's workflow, we can continuously monitor the project’s security posture. The Scorecard checks cover various security best practices and provide scores for multiple categories. Some checks include Code Reviews, Branch Protection, Signed Releases, etc. KEDA maintainers are working to improve our prevention by adding additional measures: From 1bd8b40b0e008317ea5bb3c970e63500c334078a Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 20:36:23 +0100 Subject: [PATCH 11/15] chore(deps): update github actions (#6398) | datasource | package | from | to | | ----------- | ---------------- | ------ | ------ | | github-tags | actions/checkout | v4.1.1 | v4.2.2 | Signed-off-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- .github/workflows/scorecards.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/scorecards.yml b/.github/workflows/scorecards.yml index fbb28c0e0be..00fa8661311 100644 --- a/.github/workflows/scorecards.yml +++ b/.github/workflows/scorecards.yml @@ -28,7 +28,7 @@ jobs: steps: - name: "Checkout code" - uses: actions/checkout@b4ffde65f46336ab88eb53be808477a3936bae11 # v4.1.1 + uses: actions/checkout@11bd71901bbe5b1630ceea73d27597364c9af683 # v4.2.2 with: persist-credentials: false @@ -55,7 +55,7 @@ jobs: # Upload the results as artifacts (optional). Commenting out will disable uploads of run results in SARIF # format to the repository Actions tab. - name: "Upload artifact" - uses: actions/upload-artifact@97a0fba1372883ab732affbe8f94b823f91727db # v3.pre.node20 + uses: actions/upload-artifact@ff15f0306b3f739f7b6fd43fb5d26cd321bd4de5 # v3.pre.node20 with: name: SARIF file path: results.sarif From 7e5c523681bcf03fac6eefaa336c932c8db281bd Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 20:37:36 +0100 Subject: [PATCH 12/15] chore(deps): update actions/upload-artifact action to v4 (#6399) | datasource | package | from | to | | ----------- | ----------------------- | ---- | -- | | github-tags | actions/upload-artifact | v3 | v4 | Signed-off-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- .github/workflows/scorecards.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/scorecards.yml b/.github/workflows/scorecards.yml index 00fa8661311..044bbacacf4 100644 --- a/.github/workflows/scorecards.yml +++ b/.github/workflows/scorecards.yml @@ -55,7 +55,7 @@ jobs: # Upload the results as artifacts (optional). Commenting out will disable uploads of run results in SARIF # format to the repository Actions tab. - name: "Upload artifact" - uses: actions/upload-artifact@ff15f0306b3f739f7b6fd43fb5d26cd321bd4de5 # v3.pre.node20 + uses: actions/upload-artifact@b4b15b8c7c6ac21ea08fcf65892d2ee8f75cf882 # v4.pre.node20 with: name: SARIF file path: results.sarif From c43af597fc94c8ffa8f444557d2647fc2eb1c318 Mon Sep 17 00:00:00 2001 From: Viet Nguyen Duc Date: Wed, 4 Dec 2024 08:11:43 +0000 Subject: [PATCH 13/15] fix: Selenium Grid scaler exposes sum of pending and ongoing sessions to KDEA (#6368) --- CHANGELOG.md | 1 + pkg/scalers/selenium_grid_scaler.go | 95 +++++++------ pkg/scalers/selenium_grid_scaler_test.go | 168 +++++++++++++---------- 3 files changed, 140 insertions(+), 124 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 52538a16f71..70fabdf5152 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -76,6 +76,7 @@ Here is an overview of all new **experimental** features: - **General**: Centralize and improve automaxprocs configuration with proper structured logging ([#5970](https://github.com/kedacore/keda/issues/5970)) - **General**: Paused ScaledObject count is reported correctly after operator restart ([#6321](https://github.com/kedacore/keda/issues/6321)) - **General**: ScaledJobs ready status set to true when recoverred problem ([#6329](https://github.com/kedacore/keda/pull/6329)) +- **Selenium Grid Scaler**: Exposes sum of pending and ongoing sessions to KDEA ([#6368](https://github.com/kedacore/keda/pull/6368)) ### Deprecations diff --git a/pkg/scalers/selenium_grid_scaler.go b/pkg/scalers/selenium_grid_scaler.go index 057181c87e7..bb18d7c9762 100644 --- a/pkg/scalers/selenium_grid_scaler.go +++ b/pkg/scalers/selenium_grid_scaler.go @@ -36,10 +36,10 @@ type seleniumGridScalerMetadata struct { BrowserName string `keda:"name=browserName, order=triggerMetadata"` SessionBrowserName string `keda:"name=sessionBrowserName, order=triggerMetadata, optional"` ActivationThreshold int64 `keda:"name=activationThreshold, order=triggerMetadata, optional"` - BrowserVersion string `keda:"name=browserVersion, order=triggerMetadata, optional, default=latest"` - UnsafeSsl bool `keda:"name=unsafeSsl, order=triggerMetadata, optional, default=false"` - PlatformName string `keda:"name=platformName, order=triggerMetadata, optional, default=linux"` - NodeMaxSessions int `keda:"name=nodeMaxSessions, order=triggerMetadata, optional, default=1"` + BrowserVersion string `keda:"name=browserVersion, order=triggerMetadata, default=latest"` + UnsafeSsl bool `keda:"name=unsafeSsl, order=triggerMetadata, default=false"` + PlatformName string `keda:"name=platformName, order=triggerMetadata, default=linux"` + NodeMaxSessions int64 `keda:"name=nodeMaxSessions, order=triggerMetadata, default=1"` TargetValue int64 } @@ -55,9 +55,9 @@ type Data struct { } type Grid struct { - SessionCount int `json:"sessionCount"` - MaxSession int `json:"maxSession"` - TotalSlots int `json:"totalSlots"` + SessionCount int64 `json:"sessionCount"` + MaxSession int64 `json:"maxSession"` + TotalSlots int64 `json:"totalSlots"` } type NodesInfo struct { @@ -71,17 +71,17 @@ type SessionsInfo struct { type Nodes []struct { ID string `json:"id"` Status string `json:"status"` - SessionCount int `json:"sessionCount"` - MaxSession int `json:"maxSession"` - SlotCount int `json:"slotCount"` + SessionCount int64 `json:"sessionCount"` + MaxSession int64 `json:"maxSession"` + SlotCount int64 `json:"slotCount"` Stereotypes string `json:"stereotypes"` Sessions Sessions `json:"sessions"` } type ReservedNodes struct { ID string `json:"id"` - MaxSession int `json:"maxSession"` - SlotCount int `json:"slotCount"` + MaxSession int64 `json:"maxSession"` + SlotCount int64 `json:"slotCount"` } type Sessions []struct { @@ -102,7 +102,7 @@ type Capability struct { } type Stereotypes []struct { - Slots int `json:"slots"` + Slots int64 `json:"slots"` Stereotype Capability `json:"stereotype"` } @@ -148,6 +148,7 @@ func parseSeleniumGridScalerMetadata(config *scalersconfig.ScalerConfig) (*selen if meta.SessionBrowserName == "" { meta.SessionBrowserName = meta.BrowserName } + return meta, nil } @@ -160,18 +161,18 @@ func (s *seleniumGridScaler) Close(context.Context) error { } func (s *seleniumGridScaler) GetMetricsAndActivity(ctx context.Context, metricName string) ([]external_metrics.ExternalMetricValue, bool, error) { - sessions, err := s.getSessionsCount(ctx, s.logger) + newRequestNodes, onGoingSessions, err := s.getSessionsQueueLength(ctx, s.logger) if err != nil { return []external_metrics.ExternalMetricValue{}, false, fmt.Errorf("error requesting selenium grid endpoint: %w", err) } - metric := GenerateMetricInMili(metricName, float64(sessions)) + metric := GenerateMetricInMili(metricName, float64(newRequestNodes+onGoingSessions)) - return []external_metrics.ExternalMetricValue{metric}, sessions > s.metadata.ActivationThreshold, nil + return []external_metrics.ExternalMetricValue{metric}, (newRequestNodes + onGoingSessions) > s.metadata.ActivationThreshold, nil } func (s *seleniumGridScaler) GetMetricSpecForScaling(context.Context) []v2.MetricSpec { - metricName := kedautil.NormalizeString(fmt.Sprintf("seleniumgrid-%s", s.metadata.BrowserName)) + metricName := kedautil.NormalizeString(fmt.Sprintf("selenium-grid-%s-%s-%s", s.metadata.BrowserName, s.metadata.BrowserVersion, s.metadata.PlatformName)) externalMetric := &v2.ExternalMetricSource{ Metric: v2.MetricIdentifier{ Name: GenerateMetricNameWithIndex(s.metadata.triggerIndex, metricName), @@ -184,18 +185,18 @@ func (s *seleniumGridScaler) GetMetricSpecForScaling(context.Context) []v2.Metri return []v2.MetricSpec{metricSpec} } -func (s *seleniumGridScaler) getSessionsCount(ctx context.Context, logger logr.Logger) (int64, error) { +func (s *seleniumGridScaler) getSessionsQueueLength(ctx context.Context, logger logr.Logger) (int64, int64, error) { body, err := json.Marshal(map[string]string{ "query": "{ grid { sessionCount, maxSession, totalSlots }, nodesInfo { nodes { id, status, sessionCount, maxSession, slotCount, stereotypes, sessions { id, capabilities, slot { id, stereotype } } } }, sessionsInfo { sessionQueueRequests } }", }) if err != nil { - return -1, err + return -1, -1, err } req, err := http.NewRequestWithContext(ctx, "POST", s.metadata.URL, bytes.NewBuffer(body)) if err != nil { - return -1, err + return -1, -1, err } if (s.metadata.AuthType == "" || strings.EqualFold(s.metadata.AuthType, "Basic")) && s.metadata.Username != "" && s.metadata.Password != "" { @@ -206,28 +207,28 @@ func (s *seleniumGridScaler) getSessionsCount(ctx context.Context, logger logr.L res, err := s.httpClient.Do(req) if err != nil { - return -1, err + return -1, -1, err } if res.StatusCode != http.StatusOK { msg := fmt.Sprintf("selenium grid returned %d", res.StatusCode) - return -1, errors.New(msg) + return -1, -1, errors.New(msg) } defer res.Body.Close() b, err := io.ReadAll(res.Body) if err != nil { - return -1, err + return -1, -1, err } - v, err := getCountFromSeleniumResponse(b, s.metadata.BrowserName, s.metadata.BrowserVersion, s.metadata.SessionBrowserName, s.metadata.PlatformName, s.metadata.NodeMaxSessions, logger) + newRequestNodes, onGoingSession, err := getCountFromSeleniumResponse(b, s.metadata.BrowserName, s.metadata.BrowserVersion, s.metadata.SessionBrowserName, s.metadata.PlatformName, s.metadata.NodeMaxSessions, logger) if err != nil { - return -1, err + return -1, -1, err } - return v, nil + return newRequestNodes, onGoingSession, nil } -func countMatchingSlotsStereotypes(stereotypes Stereotypes, request Capability, browserName string, browserVersion string, sessionBrowserName string, platformName string) int { - var matchingSlots int +func countMatchingSlotsStereotypes(stereotypes Stereotypes, request Capability, browserName string, browserVersion string, sessionBrowserName string, platformName string) int64 { + var matchingSlots int64 for _, stereotype := range stereotypes { if checkCapabilitiesMatch(stereotype.Stereotype, request, browserName, browserVersion, sessionBrowserName, platformName) { matchingSlots += stereotype.Slots @@ -236,8 +237,8 @@ func countMatchingSlotsStereotypes(stereotypes Stereotypes, request Capability, return matchingSlots } -func countMatchingSessions(sessions Sessions, request Capability, browserName string, browserVersion string, sessionBrowserName string, platformName string, logger logr.Logger) int { - var matchingSessions int +func countMatchingSessions(sessions Sessions, request Capability, browserName string, browserVersion string, sessionBrowserName string, platformName string, logger logr.Logger) int64 { + var matchingSessions int64 for _, session := range sessions { var capability = Capability{} if err := json.Unmarshal([]byte(session.Capabilities), &capability); err == nil { @@ -274,7 +275,7 @@ func checkCapabilitiesMatch(capability Capability, requestCapability Capability, return browserNameMatches && browserVersionMatches && platformNameMatches } -func checkNodeReservedSlots(reservedNodes []ReservedNodes, nodeID string, availableSlots int) int { +func checkNodeReservedSlots(reservedNodes []ReservedNodes, nodeID string, availableSlots int64) int64 { for _, reservedNode := range reservedNodes { if strings.EqualFold(reservedNode.ID, nodeID) { return reservedNode.SlotCount @@ -283,7 +284,7 @@ func checkNodeReservedSlots(reservedNodes []ReservedNodes, nodeID string, availa return availableSlots } -func updateOrAddReservedNode(reservedNodes []ReservedNodes, nodeID string, slotCount int, maxSession int) []ReservedNodes { +func updateOrAddReservedNode(reservedNodes []ReservedNodes, nodeID string, slotCount int64, maxSession int64) []ReservedNodes { for i, reservedNode := range reservedNodes { if strings.EqualFold(reservedNode.ID, nodeID) { // Update remaining available slots for the reserved node @@ -295,17 +296,15 @@ func updateOrAddReservedNode(reservedNodes []ReservedNodes, nodeID string, slotC return append(reservedNodes, ReservedNodes{ID: nodeID, SlotCount: slotCount, MaxSession: maxSession}) } -func getCountFromSeleniumResponse(b []byte, browserName string, browserVersion string, sessionBrowserName string, platformName string, nodeMaxSessions int, logger logr.Logger) (int64, error) { - // The returned count of the number of new Nodes will be scaled up - var count int64 +func getCountFromSeleniumResponse(b []byte, browserName string, browserVersion string, sessionBrowserName string, platformName string, nodeMaxSessions int64, logger logr.Logger) (int64, int64, error) { // Track number of available slots of existing Nodes in the Grid can be reserved for the matched requests - var availableSlots int + var availableSlots int64 // Track number of matched requests in the sessions queue will be served by this scaler - var queueSlots int + var queueSlots int64 var seleniumResponse = SeleniumResponse{} if err := json.Unmarshal(b, &seleniumResponse); err != nil { - return 0, err + return 0, 0, err } var sessionQueueRequests = seleniumResponse.Data.SessionsInfo.SessionQueueRequests @@ -314,6 +313,7 @@ func getCountFromSeleniumResponse(b []byte, browserName string, browserVersion s var reservedNodes []ReservedNodes // Track list of new Nodes will be scaled up with number of available slots following scaler parameter `nodeMaxSessions` var newRequestNodes []ReservedNodes + var onGoingSessions int64 for requestIndex, sessionQueueRequest := range sessionQueueRequests { var isRequestMatched bool var requestCapability = Capability{} @@ -332,20 +332,22 @@ func getCountFromSeleniumResponse(b []byte, browserName string, browserVersion s } var isRequestReserved bool + var sumOfCurrentSessionsMatch int64 // Check if the matched request can be assigned to available slots of existing Nodes in the Grid for _, node := range nodes { + // Count ongoing sessions that match the request capability and scaler metadata + var currentSessionsMatch = countMatchingSessions(node.Sessions, requestCapability, browserName, browserVersion, sessionBrowserName, platformName, logger) + sumOfCurrentSessionsMatch += currentSessionsMatch // Check if node is UP and has available slots (maxSession > sessionCount) if strings.EqualFold(node.Status, "UP") && checkNodeReservedSlots(reservedNodes, node.ID, node.MaxSession-node.SessionCount) > 0 { var stereotypes = Stereotypes{} - var availableSlotsMatch int + var availableSlotsMatch int64 if err := json.Unmarshal([]byte(node.Stereotypes), &stereotypes); err == nil { // Count available slots that match the request capability and scaler metadata availableSlotsMatch += countMatchingSlotsStereotypes(stereotypes, requestCapability, browserName, browserVersion, sessionBrowserName, platformName) } else { logger.Error(err, fmt.Sprintf("Error when unmarshaling node stereotypes: %s", err)) } - // Count ongoing sessions that match the request capability and scaler metadata - var currentSessionsMatch = countMatchingSessions(node.Sessions, requestCapability, browserName, browserVersion, sessionBrowserName, platformName, logger) // Count remaining available slots can be reserved for this request var availableSlotsCanBeReserved = checkNodeReservedSlots(reservedNodes, node.ID, node.MaxSession-node.SessionCount) // Reserve one available slot for the request if available slots match is greater than current sessions match @@ -357,6 +359,9 @@ func getCountFromSeleniumResponse(b []byte, browserName string, browserVersion s } } } + if sumOfCurrentSessionsMatch > onGoingSessions { + onGoingSessions = sumOfCurrentSessionsMatch + } // Check if the matched request can be assigned to available slots of new Nodes will be scaled up, since the scaler parameter `nodeMaxSessions` can be greater than 1 if !isRequestReserved { for _, newRequestNode := range newRequestNodes { @@ -373,11 +378,5 @@ func getCountFromSeleniumResponse(b []byte, browserName string, browserVersion s } } - if queueSlots > availableSlots { - count = int64(len(newRequestNodes)) - } else { - count = 0 - } - - return count, nil + return int64(len(newRequestNodes)), onGoingSessions, nil } diff --git a/pkg/scalers/selenium_grid_scaler_test.go b/pkg/scalers/selenium_grid_scaler_test.go index b92936cbe7f..6613be242ca 100644 --- a/pkg/scalers/selenium_grid_scaler_test.go +++ b/pkg/scalers/selenium_grid_scaler_test.go @@ -16,13 +16,14 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { sessionBrowserName string browserVersion string platformName string - nodeMaxSessions int + nodeMaxSessions int64 } tests := []struct { - name string - args args - want int64 - wantErr bool + name string + args args + wantNewRequestNodes int64 + wantOnGoingSessions int64 + wantErr bool }{ { name: "nil response body should throw error", @@ -61,8 +62,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { `), browserName: "", }, - want: 0, - wantErr: false, + wantNewRequestNodes: 0, + wantErr: false, }, { name: "12 sessionQueueRequests with 4 requests matching browserName chrome should return count as 4", @@ -101,8 +102,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 4, - wantErr: false, + wantNewRequestNodes: 4, + wantErr: false, }, { name: "2 sessionQueueRequests and 1 available nodeStereotypes with matching browserName firefox should return count as 1", @@ -276,8 +277,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantOnGoingSessions: 4, + wantErr: false, }, { name: "1 sessionQueueRequests and 1 available nodeStereotypes with matching browserName chrome should return count as 0", @@ -325,8 +327,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 0, - wantErr: false, + wantNewRequestNodes: 0, + wantErr: false, }, { name: "1 sessionQueueRequests Linux and 1 available nodeStereotypes Windows with matching browserName chrome should return count as 1", @@ -374,8 +376,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantErr: false, }, { name: "scaler browserVersion is latest, 2 sessionQueueRequests wihtout browserVersion, 2 available nodeStereotypes with different versions and platforms, should return count as 1", @@ -422,8 +424,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantErr: false, }, { name: "scaler browserVersion is latest, 5 sessionQueueRequests wihtout browserVersion also 1 different platformName, 1 available nodeStereotypes with 3 slots Linux and 1 node Windows, should return count as 1", @@ -473,8 +475,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantErr: false, }, { name: "queue request with browserName browserVersion and browserVersion but no available nodes should return count as 1", @@ -516,8 +518,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantErr: false, }, { name: "1 queue request with browserName browserVersion and browserVersion but 2 nodes without available slots should return count as 1", @@ -573,8 +575,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantOnGoingSessions: 2, + wantErr: false, }, { name: "2 session queue with matching browsername and browserversion of 2 available slots should return count as 0", @@ -621,8 +624,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "linux", }, - want: 0, - wantErr: false, + wantNewRequestNodes: 0, + wantErr: false, }, { name: "2 queue requests with browserName browserVersion and platformName matching 2 available slots on 2 different nodes should return count as 0", @@ -679,8 +682,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "linux", }, - want: 0, - wantErr: false, + wantNewRequestNodes: 0, + wantOnGoingSessions: 2, + wantErr: false, }, { name: "1 queue request with browserName browserVersion and platformName matching 1 available slot on node has 3 max sessions should return count as 0", @@ -726,8 +730,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "linux", }, - want: 0, - wantErr: false, + wantNewRequestNodes: 0, + wantOnGoingSessions: 2, + wantErr: false, }, { name: "3 queue requests with browserName browserVersion and platformName but 2 running nodes are busy should return count as 3", @@ -785,8 +790,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "linux", }, - want: 3, - wantErr: false, + wantNewRequestNodes: 3, + wantOnGoingSessions: 2, + wantErr: false, }, { name: "3 queue requests with browserName browserVersion and platformName but 2 running nodes are busy with different versions should return count as 3", @@ -844,8 +850,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 3, - wantErr: false, + wantNewRequestNodes: 3, + wantOnGoingSessions: 2, + wantErr: false, }, { name: "3 queue requests with browserName and platformName but 2 running nodes are busy with different versions should return count as 3", @@ -903,8 +910,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 3, - wantErr: false, + wantNewRequestNodes: 3, + wantOnGoingSessions: 2, + wantErr: false, }, { name: "1 active session with matching browsername and version should return count as 2", @@ -947,8 +955,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "linux", }, - want: 2, - wantErr: false, + wantNewRequestNodes: 2, + wantOnGoingSessions: 1, + wantErr: false, }, { name: "1 request without browserName and browserVersion stable can be match any available node should return count as 0", @@ -985,8 +994,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 0, - wantErr: false, + wantNewRequestNodes: 0, + wantErr: false, }, { name: "1 request without browserName and browserVersion stable should return count as 1", @@ -1028,8 +1037,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantOnGoingSessions: 1, + wantErr: false, }, { name: "2 queue requests with browserName in string match node stereotype and scaler metadata browserVersion should return count as 1", @@ -1072,8 +1082,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "dev", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantOnGoingSessions: 1, + wantErr: false, }, { name: "2 queue requests with matching browsername/sessionBrowserName but 1 node is busy should return count as 2", @@ -1116,8 +1127,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "linux", }, - want: 2, - wantErr: false, + wantNewRequestNodes: 2, + wantOnGoingSessions: 1, + wantErr: false, }, { name: "2 queue requests with matching browsername/sessionBrowserName and 1 node is is available should return count as 1", @@ -1155,8 +1167,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantErr: false, }, { name: "2 queue requests with platformName and without platformName and node with 1 slot available should return count as 1", args: args{ @@ -1198,8 +1210,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "Windows 11", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantOnGoingSessions: 1, + wantErr: false, }, { name: "1 active msedge session while asking for 2 chrome sessions should return a count of 2", @@ -1242,8 +1255,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 2, - wantErr: false, + wantNewRequestNodes: 2, + wantErr: false, }, { name: "3 queue requests browserName chrome platformName linux but 1 node has maxSessions=3 with browserName msedge should return a count of 3", @@ -1287,8 +1300,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 3, - wantErr: false, + wantNewRequestNodes: 3, + wantErr: false, }, { name: "session request with matching browsername and no specific platformName should return count as 2", @@ -1316,8 +1329,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "", }, - want: 2, - wantErr: false, + wantNewRequestNodes: 2, + wantErr: false, }, { name: "2 queue requests with 1 matching browsername and platformName and 1 existing slot is available should return count as 0", @@ -1355,8 +1368,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "Windows 11", }, - want: 0, - wantErr: false, + wantNewRequestNodes: 0, + wantErr: false, }, { name: "2 queue requests with 1 request matching browserName and platformName but 1 existing node is busy should return count as 1", @@ -1403,8 +1416,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0", platformName: "Windows 11", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantOnGoingSessions: 1, + wantErr: false, }, { name: "5 queue requests with scaler parameter nodeMaxSessions is 2 should return count as 3", @@ -1437,8 +1451,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { platformName: "linux", nodeMaxSessions: 2, }, - want: 3, - wantErr: false, + wantNewRequestNodes: 3, + wantErr: false, }, { name: "5 queue requests with scaler parameter nodeMaxSessions is 3 should return count as 2", @@ -1471,8 +1485,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { platformName: "linux", nodeMaxSessions: 3, }, - want: 2, - wantErr: false, + wantNewRequestNodes: 2, + wantErr: false, }, { name: "5 queue requests with request matching browserName and platformName and scaler param nodeMaxSessions is 3 and existing node with 1 available slot should return count as 2", @@ -1523,8 +1537,9 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { platformName: "linux", nodeMaxSessions: 3, }, - want: 2, - wantErr: false, + wantNewRequestNodes: 2, + wantOnGoingSessions: 2, + wantErr: false, }, // Tests from PR: https://github.com/kedacore/keda/pull/6055 { @@ -1563,8 +1578,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 0, - wantErr: false, + wantNewRequestNodes: 0, + wantErr: false, }, { name: "4 sessions requests with matching browsername and platformName when setSessionsFromHub turned on and node with 2 slots matches should return count as 2", @@ -1605,8 +1620,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "latest", platformName: "linux", }, - want: 2, - wantErr: false, + wantNewRequestNodes: 2, + wantErr: false, }, { name: "4 sessions requests with matching browsername and platformName when setSessionsFromHub turned on, no nodes and sessionsPerNode=2 matches should return count as 2", @@ -1637,8 +1652,8 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { platformName: "linux", nodeMaxSessions: 2, }, - want: 2, - wantErr: false, + wantNewRequestNodes: 2, + wantErr: false, }, { name: "sessions requests and active sessions with 1 matching browsername, platformName and sessionBrowserVersion should return count as 1", @@ -1687,19 +1702,20 @@ func Test_getCountFromSeleniumResponse(t *testing.T) { browserVersion: "91.0.4472.114", platformName: "linux", }, - want: 1, - wantErr: false, + wantNewRequestNodes: 1, + wantOnGoingSessions: 2, + wantErr: false, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, err := getCountFromSeleniumResponse(tt.args.b, tt.args.browserName, tt.args.browserVersion, tt.args.sessionBrowserName, tt.args.platformName, tt.args.nodeMaxSessions, logr.Discard()) + newRequestNodes, onGoingSessions, err := getCountFromSeleniumResponse(tt.args.b, tt.args.browserName, tt.args.browserVersion, tt.args.sessionBrowserName, tt.args.platformName, tt.args.nodeMaxSessions, logr.Discard()) if (err != nil) != tt.wantErr { t.Errorf("getCountFromSeleniumResponse() error = %v, wantErr %v", err, tt.wantErr) return } - if !reflect.DeepEqual(got, tt.want) { - t.Errorf("getCountFromSeleniumResponse() = %v, want %v", got, tt.want) + if !reflect.DeepEqual(newRequestNodes, tt.wantNewRequestNodes) || !reflect.DeepEqual(onGoingSessions, tt.wantOnGoingSessions) { + t.Errorf("getCountFromSeleniumResponse() = [%v, %v], want [%v, %v]", newRequestNodes, onGoingSessions, tt.wantNewRequestNodes, tt.wantOnGoingSessions) } }) } From 1eaa34c724b1d1c85d28971c7bfc76b7d6bd5591 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Maksymilian=20Bogu=C5=84?= Date: Wed, 4 Dec 2024 09:44:22 +0100 Subject: [PATCH 14/15] Add AWS region to the AWS Config Cache key (#6134) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Introduce aws region into the AWS config cache Signed-off-by: Maksymilian Boguń * add CHANGELOG entry Signed-off-by: Maksymilian Boguń * embedded AWS region into Authorization metadata Signed-off-by: Maksymilian Boguń * move the fix to Unreleased version Signed-off-by: Maksymilian Boguń * Fix indentation Signed-off-by: Maksymilian Boguń --------- Signed-off-by: Maksymilian Boguń Signed-off-by: Jan Wozniak Co-authored-by: Jorge Turrado Ferrero Co-authored-by: Jan Wozniak --- CHANGELOG.md | 1 + pkg/scalers/apache_kafka_scaler.go | 2 +- pkg/scalers/aws/aws_authorization.go | 2 + pkg/scalers/aws/aws_common.go | 31 ++++---- pkg/scalers/aws/aws_config_cache.go | 10 +-- pkg/scalers/aws/aws_config_cache_test.go | 77 +++++++++++-------- pkg/scalers/aws/aws_sigv4.go | 18 ++--- pkg/scalers/aws_cloudwatch_scaler.go | 2 +- pkg/scalers/aws_dynamodb_scaler.go | 2 +- pkg/scalers/aws_dynamodb_streams_scaler.go | 2 +- pkg/scalers/aws_kinesis_stream_scaler.go | 2 +- pkg/scalers/aws_sqs_queue_scaler.go | 2 +- pkg/scalers/kafka_scaler.go | 2 +- .../resolver/aws_secretmanager_handler.go | 3 +- 14 files changed, 79 insertions(+), 77 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 70fabdf5152..266aea0115b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -76,6 +76,7 @@ Here is an overview of all new **experimental** features: - **General**: Centralize and improve automaxprocs configuration with proper structured logging ([#5970](https://github.com/kedacore/keda/issues/5970)) - **General**: Paused ScaledObject count is reported correctly after operator restart ([#6321](https://github.com/kedacore/keda/issues/6321)) - **General**: ScaledJobs ready status set to true when recoverred problem ([#6329](https://github.com/kedacore/keda/pull/6329)) +- **AWS Scalers**: Add AWS region to the AWS Config Cache key ([#6128](https://github.com/kedacore/keda/issues/6128)) - **Selenium Grid Scaler**: Exposes sum of pending and ongoing sessions to KDEA ([#6368](https://github.com/kedacore/keda/pull/6368)) ### Deprecations diff --git a/pkg/scalers/apache_kafka_scaler.go b/pkg/scalers/apache_kafka_scaler.go index 8704531e28a..95ca5faf155 100644 --- a/pkg/scalers/apache_kafka_scaler.go +++ b/pkg/scalers/apache_kafka_scaler.go @@ -220,7 +220,7 @@ func getApacheKafkaClient(ctx context.Context, metadata apacheKafkaMetadata, log case KafkaSASLTypeOAuthbearer: return nil, errors.New("SASL/OAUTHBEARER is not implemented yet") case KafkaSASLTypeMskIam: - cfg, err := awsutils.GetAwsConfig(ctx, metadata.AWSRegion, metadata.AWSAuthorization) + cfg, err := awsutils.GetAwsConfig(ctx, metadata.AWSAuthorization) if err != nil { return nil, err } diff --git a/pkg/scalers/aws/aws_authorization.go b/pkg/scalers/aws/aws_authorization.go index fd49c2f8995..8fed034553c 100644 --- a/pkg/scalers/aws/aws_authorization.go +++ b/pkg/scalers/aws/aws_authorization.go @@ -23,6 +23,8 @@ type AuthorizationMetadata struct { AwsSecretAccessKey string AwsSessionToken string + AwsRegion string + // Deprecated PodIdentityOwner bool // Pod identity owner is confusing and it'll be removed when we get diff --git a/pkg/scalers/aws/aws_common.go b/pkg/scalers/aws/aws_common.go index ed98e2b219d..30f25e98a4a 100644 --- a/pkg/scalers/aws/aws_common.go +++ b/pkg/scalers/aws/aws_common.go @@ -39,43 +39,33 @@ import ( // ErrAwsNoAccessKey is returned when awsAccessKeyID is missing. var ErrAwsNoAccessKey = errors.New("awsAccessKeyID not found") -type awsConfigMetadata struct { - awsRegion string - awsAuthorization AuthorizationMetadata -} - var awsSharedCredentialsCache = newSharedConfigsCache() // GetAwsConfig returns an *aws.Config for a given AuthorizationMetadata // If AuthorizationMetadata uses static credentials or `aws` auth, // we recover the *aws.Config from the shared cache. If not, we generate // a new entry on each request -func GetAwsConfig(ctx context.Context, awsRegion string, awsAuthorization AuthorizationMetadata) (*aws.Config, error) { - metadata := &awsConfigMetadata{ - awsRegion: awsRegion, - awsAuthorization: awsAuthorization, - } - - if metadata.awsAuthorization.UsingPodIdentity || - (metadata.awsAuthorization.AwsAccessKeyID != "" && metadata.awsAuthorization.AwsSecretAccessKey != "") { - return awsSharedCredentialsCache.GetCredentials(ctx, metadata.awsRegion, metadata.awsAuthorization) +func GetAwsConfig(ctx context.Context, awsAuthorization AuthorizationMetadata) (*aws.Config, error) { + if awsAuthorization.UsingPodIdentity || + (awsAuthorization.AwsAccessKeyID != "" && awsAuthorization.AwsSecretAccessKey != "") { + return awsSharedCredentialsCache.GetCredentials(ctx, awsAuthorization) } // TODO, remove when aws-eks are removed configOptions := make([]func(*config.LoadOptions) error, 0) - configOptions = append(configOptions, config.WithRegion(metadata.awsRegion)) + configOptions = append(configOptions, config.WithRegion(awsAuthorization.AwsRegion)) cfg, err := config.LoadDefaultConfig(ctx, configOptions...) if err != nil { return nil, err } - if !metadata.awsAuthorization.PodIdentityOwner { + if !awsAuthorization.PodIdentityOwner { return &cfg, nil } - if metadata.awsAuthorization.AwsRoleArn != "" { + if awsAuthorization.AwsRoleArn != "" { stsSvc := sts.NewFromConfig(cfg) - stsCredentialProvider := stscreds.NewAssumeRoleProvider(stsSvc, metadata.awsAuthorization.AwsRoleArn, func(_ *stscreds.AssumeRoleOptions) {}) + stsCredentialProvider := stscreds.NewAssumeRoleProvider(stsSvc, awsAuthorization.AwsRoleArn, func(_ *stscreds.AssumeRoleOptions) {}) cfg.Credentials = aws.NewCredentialsCache(stsCredentialProvider) } return &cfg, err @@ -88,6 +78,10 @@ func GetAwsAuthorization(uniqueKey string, podIdentity kedav1alpha1.AuthPodIdent TriggerUniqueKey: uniqueKey, } + if val, ok := authParams["awsRegion"]; ok && val != "" { + meta.AwsRegion = val + } + if podIdentity.Provider == kedav1alpha1.PodIdentityProviderAws { meta.UsingPodIdentity = true if val, ok := authParams["awsRoleArn"]; ok && val != "" { @@ -95,6 +89,7 @@ func GetAwsAuthorization(uniqueKey string, podIdentity kedav1alpha1.AuthPodIdent } return meta, nil } + // TODO, remove all the logic below and just keep the logic for // parsing awsAccessKeyID, awsSecretAccessKey and awsSessionToken // when aws-eks are removed diff --git a/pkg/scalers/aws/aws_config_cache.go b/pkg/scalers/aws/aws_config_cache.go index 684e45c743b..b43056b423b 100644 --- a/pkg/scalers/aws/aws_config_cache.go +++ b/pkg/scalers/aws/aws_config_cache.go @@ -69,11 +69,11 @@ func newSharedConfigsCache() sharedConfigCache { // getCacheKey returns a unique key based on given AuthorizationMetadata. // As it can contain sensitive data, the key is hashed to not expose secrets func (a *sharedConfigCache) getCacheKey(awsAuthorization AuthorizationMetadata) string { - key := "keda" + key := "keda-" + awsAuthorization.AwsRegion if awsAuthorization.AwsAccessKeyID != "" { - key = fmt.Sprintf("%s-%s-%s", awsAuthorization.AwsAccessKeyID, awsAuthorization.AwsSecretAccessKey, awsAuthorization.AwsSessionToken) + key = fmt.Sprintf("%s-%s-%s-%s", awsAuthorization.AwsAccessKeyID, awsAuthorization.AwsSecretAccessKey, awsAuthorization.AwsSessionToken, awsAuthorization.AwsRegion) } else if awsAuthorization.AwsRoleArn != "" { - key = awsAuthorization.AwsRoleArn + key = fmt.Sprintf("%s-%s", awsAuthorization.AwsRoleArn, awsAuthorization.AwsRegion) } // to avoid sensitive data as key and to use a constant key size, // we hash the key with sha3 @@ -86,7 +86,7 @@ func (a *sharedConfigCache) getCacheKey(awsAuthorization AuthorizationMetadata) // sharing it between all the requests. To track if the *aws.Config is used by whom, // every time when an scaler requests *aws.Config we register it inside // the cached item. -func (a *sharedConfigCache) GetCredentials(ctx context.Context, awsRegion string, awsAuthorization AuthorizationMetadata) (*aws.Config, error) { +func (a *sharedConfigCache) GetCredentials(ctx context.Context, awsAuthorization AuthorizationMetadata) (*aws.Config, error) { a.Lock() defer a.Unlock() key := a.getCacheKey(awsAuthorization) @@ -97,7 +97,7 @@ func (a *sharedConfigCache) GetCredentials(ctx context.Context, awsRegion string } configOptions := make([]func(*config.LoadOptions) error, 0) - configOptions = append(configOptions, config.WithRegion(awsRegion)) + configOptions = append(configOptions, config.WithRegion(awsAuthorization.AwsRegion)) cfg, err := config.LoadDefaultConfig(ctx, configOptions...) if err != nil { return nil, err diff --git a/pkg/scalers/aws/aws_config_cache_test.go b/pkg/scalers/aws/aws_config_cache_test.go index d94247a6fee..81abbdca6c7 100644 --- a/pkg/scalers/aws/aws_config_cache_test.go +++ b/pkg/scalers/aws/aws_config_cache_test.go @@ -28,84 +28,95 @@ import ( func TestGetCredentialsReturnNewItemAndStoreItIfNotExist(t *testing.T) { cache := newSharedConfigsCache() cache.logger = logr.Discard() - config := awsConfigMetadata{ - awsRegion: "test-region", - awsAuthorization: AuthorizationMetadata{ - TriggerUniqueKey: "test-key", - }, + awsAuthorization := AuthorizationMetadata{ + TriggerUniqueKey: "test-key", + AwsRegion: "test-region", } - cacheKey := cache.getCacheKey(config.awsAuthorization) - _, err := cache.GetCredentials(context.Background(), config.awsRegion, config.awsAuthorization) + cacheKey := cache.getCacheKey(awsAuthorization) + _, err := cache.GetCredentials(context.Background(), awsAuthorization) assert.NoError(t, err) assert.Contains(t, cache.items, cacheKey) - assert.Contains(t, cache.items[cacheKey].usages, config.awsAuthorization.TriggerUniqueKey) + assert.Contains(t, cache.items[cacheKey].usages, awsAuthorization.TriggerUniqueKey) } func TestGetCredentialsReturnCachedItemIfExist(t *testing.T) { cache := newSharedConfigsCache() cache.logger = logr.Discard() - config := awsConfigMetadata{ - awsRegion: "test1-region", - awsAuthorization: AuthorizationMetadata{ - TriggerUniqueKey: "test1-key", - }, + awsAuthorization := AuthorizationMetadata{ + TriggerUniqueKey: "test1-key", + AwsRegion: "test1-region", } cfg := aws.Config{} cfg.AppID = "test1-app" - cacheKey := cache.getCacheKey(config.awsAuthorization) + cacheKey := cache.getCacheKey(awsAuthorization) cache.items[cacheKey] = cacheEntry{ config: &cfg, usages: map[string]bool{ "other-usage": true, }, } - configFromCache, err := cache.GetCredentials(context.Background(), config.awsRegion, config.awsAuthorization) + configFromCache, err := cache.GetCredentials(context.Background(), awsAuthorization) assert.NoError(t, err) assert.Equal(t, &cfg, configFromCache) - assert.Contains(t, cache.items[cacheKey].usages, config.awsAuthorization.TriggerUniqueKey) + assert.Contains(t, cache.items[cacheKey].usages, awsAuthorization.TriggerUniqueKey) } func TestRemoveCachedEntryRemovesCachedItemIfNotUsages(t *testing.T) { cache := newSharedConfigsCache() cache.logger = logr.Discard() - config := awsConfigMetadata{ - awsRegion: "test2-region", - awsAuthorization: AuthorizationMetadata{ - TriggerUniqueKey: "test2-key", - }, + awsAuthorization := AuthorizationMetadata{ + TriggerUniqueKey: "test2-key", + AwsRegion: "test2-region", } cfg := aws.Config{} cfg.AppID = "test2-app" - cacheKey := cache.getCacheKey(config.awsAuthorization) + cacheKey := cache.getCacheKey(awsAuthorization) cache.items[cacheKey] = cacheEntry{ config: &cfg, usages: map[string]bool{ - config.awsAuthorization.TriggerUniqueKey: true, + awsAuthorization.TriggerUniqueKey: true, }, } - cache.RemoveCachedEntry(config.awsAuthorization) + cache.RemoveCachedEntry(awsAuthorization) assert.NotContains(t, cache.items, cacheKey) } func TestRemoveCachedEntryNotRemoveCachedItemIfUsages(t *testing.T) { cache := newSharedConfigsCache() cache.logger = logr.Discard() - config := awsConfigMetadata{ - awsRegion: "test3-region", - awsAuthorization: AuthorizationMetadata{ - TriggerUniqueKey: "test3-key", - }, + awsAuthorization := AuthorizationMetadata{ + TriggerUniqueKey: "test3-key", + AwsRegion: "test3-region", } cfg := aws.Config{} cfg.AppID = "test3-app" - cacheKey := cache.getCacheKey(config.awsAuthorization) + cacheKey := cache.getCacheKey(awsAuthorization) cache.items[cacheKey] = cacheEntry{ config: &cfg, usages: map[string]bool{ - config.awsAuthorization.TriggerUniqueKey: true, - "other-usage": true, + awsAuthorization.TriggerUniqueKey: true, + "other-usage": true, }, } - cache.RemoveCachedEntry(config.awsAuthorization) + cache.RemoveCachedEntry(awsAuthorization) assert.Contains(t, cache.items, cacheKey) } + +func TestCredentialsShouldBeCachedPerRegion(t *testing.T) { + cache := newSharedConfigsCache() + cache.logger = logr.Discard() + awsAuthorization1 := AuthorizationMetadata{ + TriggerUniqueKey: "test4-key", + AwsRegion: "test4-region1", + } + awsAuthorization2 := AuthorizationMetadata{ + TriggerUniqueKey: "test4-key", + AwsRegion: "test4-region2", + } + cred1, err1 := cache.GetCredentials(context.Background(), awsAuthorization1) + cred2, err2 := cache.GetCredentials(context.Background(), awsAuthorization2) + + assert.NoError(t, err1) + assert.NoError(t, err2) + assert.NotEqual(t, cred1, cred2, "Credentials should be stored per region") +} diff --git a/pkg/scalers/aws/aws_sigv4.go b/pkg/scalers/aws/aws_sigv4.go index 2abde772f1c..0be031938b7 100644 --- a/pkg/scalers/aws/aws_sigv4.go +++ b/pkg/scalers/aws/aws_sigv4.go @@ -71,20 +71,12 @@ func (rt *roundTripper) RoundTrip(req *http.Request) (*http.Response, error) { } // parseAwsAMPMetadata parses the data to get the AWS sepcific auth info and metadata -func parseAwsAMPMetadata(config *scalersconfig.ScalerConfig) (*awsConfigMetadata, error) { - meta := awsConfigMetadata{} - - if val, ok := config.TriggerMetadata["awsRegion"]; ok && val != "" { - meta.awsRegion = val - } - +func parseAwsAMPMetadata(config *scalersconfig.ScalerConfig) (*AuthorizationMetadata, error) { auth, err := GetAwsAuthorization(config.TriggerUniqueKey, config.PodIdentity, config.TriggerMetadata, config.AuthParams, config.ResolvedEnv) if err != nil { return nil, err } - - meta.awsAuthorization = auth - return &meta, nil + return &auth, nil } // NewSigV4RoundTripper returns a new http.RoundTripper that will sign requests @@ -100,11 +92,11 @@ func NewSigV4RoundTripper(config *scalersconfig.ScalerConfig) (http.RoundTripper // which is probably the reason to create a SigV4RoundTripper. // To prevent failures we check if the metadata is nil // (missing AWS info) and we hide the error - metadata, _ := parseAwsAMPMetadata(config) - if metadata == nil { + awsAuthorization, _ := parseAwsAMPMetadata(config) + if awsAuthorization == nil { return nil, nil } - awsCfg, err := GetAwsConfig(context.Background(), metadata.awsRegion, metadata.awsAuthorization) + awsCfg, err := GetAwsConfig(context.Background(), *awsAuthorization) if err != nil { return nil, err } diff --git a/pkg/scalers/aws_cloudwatch_scaler.go b/pkg/scalers/aws_cloudwatch_scaler.go index 9eacc785714..ee2483e898f 100644 --- a/pkg/scalers/aws_cloudwatch_scaler.go +++ b/pkg/scalers/aws_cloudwatch_scaler.go @@ -115,7 +115,7 @@ func NewAwsCloudwatchScaler(ctx context.Context, config *scalersconfig.ScalerCon } func createCloudwatchClient(ctx context.Context, metadata *awsCloudwatchMetadata) (*cloudwatch.Client, error) { - cfg, err := awsutils.GetAwsConfig(ctx, metadata.AwsRegion, metadata.awsAuthorization) + cfg, err := awsutils.GetAwsConfig(ctx, metadata.awsAuthorization) if err != nil { return nil, err diff --git a/pkg/scalers/aws_dynamodb_scaler.go b/pkg/scalers/aws_dynamodb_scaler.go index f3c9bcac1f6..9668ff07ea7 100644 --- a/pkg/scalers/aws_dynamodb_scaler.go +++ b/pkg/scalers/aws_dynamodb_scaler.go @@ -125,7 +125,7 @@ func parseAwsDynamoDBMetadata(config *scalersconfig.ScalerConfig) (*awsDynamoDBM } func createDynamoDBClient(ctx context.Context, metadata *awsDynamoDBMetadata) (*dynamodb.Client, error) { - cfg, err := awsutils.GetAwsConfig(ctx, metadata.AwsRegion, metadata.awsAuthorization) + cfg, err := awsutils.GetAwsConfig(ctx, metadata.awsAuthorization) if err != nil { return nil, err } diff --git a/pkg/scalers/aws_dynamodb_streams_scaler.go b/pkg/scalers/aws_dynamodb_streams_scaler.go index cdcd8548320..57e49821eff 100644 --- a/pkg/scalers/aws_dynamodb_streams_scaler.go +++ b/pkg/scalers/aws_dynamodb_streams_scaler.go @@ -92,7 +92,7 @@ func parseAwsDynamoDBStreamsMetadata(config *scalersconfig.ScalerConfig) (*awsDy } func createClientsForDynamoDBStreamsScaler(ctx context.Context, metadata *awsDynamoDBStreamsMetadata) (*dynamodb.Client, *dynamodbstreams.Client, error) { - cfg, err := awsutils.GetAwsConfig(ctx, metadata.AwsRegion, metadata.awsAuthorization) + cfg, err := awsutils.GetAwsConfig(ctx, metadata.awsAuthorization) if err != nil { return nil, nil, err } diff --git a/pkg/scalers/aws_kinesis_stream_scaler.go b/pkg/scalers/aws_kinesis_stream_scaler.go index 65fb90a4e23..efbb2d4cb32 100644 --- a/pkg/scalers/aws_kinesis_stream_scaler.go +++ b/pkg/scalers/aws_kinesis_stream_scaler.go @@ -131,7 +131,7 @@ func parseAwsKinesisStreamMetadata(config *scalersconfig.ScalerConfig, logger lo } func createKinesisClient(ctx context.Context, metadata *awsKinesisStreamMetadata) (*kinesis.Client, error) { - cfg, err := awsutils.GetAwsConfig(ctx, metadata.awsRegion, metadata.awsAuthorization) + cfg, err := awsutils.GetAwsConfig(ctx, metadata.awsAuthorization) if err != nil { return nil, err } diff --git a/pkg/scalers/aws_sqs_queue_scaler.go b/pkg/scalers/aws_sqs_queue_scaler.go index 6f1b6b5d0ee..457be4522ad 100644 --- a/pkg/scalers/aws_sqs_queue_scaler.go +++ b/pkg/scalers/aws_sqs_queue_scaler.go @@ -122,7 +122,7 @@ func parseAwsSqsQueueMetadata(config *scalersconfig.ScalerConfig) (*awsSqsQueueM } func createSqsClient(ctx context.Context, metadata *awsSqsQueueMetadata) (*sqs.Client, error) { - cfg, err := awsutils.GetAwsConfig(ctx, metadata.AwsRegion, metadata.awsAuthorization) + cfg, err := awsutils.GetAwsConfig(ctx, metadata.awsAuthorization) if err != nil { return nil, err } diff --git a/pkg/scalers/kafka_scaler.go b/pkg/scalers/kafka_scaler.go index b353c1313b4..5b1adaf77ad 100644 --- a/pkg/scalers/kafka_scaler.go +++ b/pkg/scalers/kafka_scaler.go @@ -669,7 +669,7 @@ func getKafkaClientConfig(ctx context.Context, metadata kafkaMetadata) (*sarama. case KafkaSASLOAuthTokenProviderBearer: config.Net.SASL.TokenProvider = kafka.OAuthBearerTokenProvider(metadata.username, metadata.password, metadata.oauthTokenEndpointURI, metadata.scopes, metadata.oauthExtensions) case KafkaSASLOAuthTokenProviderAWSMSKIAM: - awsAuth, err := awsutils.GetAwsConfig(ctx, metadata.awsRegion, metadata.awsAuthorization) + awsAuth, err := awsutils.GetAwsConfig(ctx, metadata.awsAuthorization) if err != nil { return nil, fmt.Errorf("error getting AWS config: %w", err) } diff --git a/pkg/scaling/resolver/aws_secretmanager_handler.go b/pkg/scaling/resolver/aws_secretmanager_handler.go index 6c07281262a..595ffb4c6f1 100644 --- a/pkg/scaling/resolver/aws_secretmanager_handler.go +++ b/pkg/scaling/resolver/aws_secretmanager_handler.go @@ -73,6 +73,7 @@ func (ash *AwsSecretManagerHandler) Initialize(ctx context.Context, client clien if ash.secretManager.Region != "" { awsRegion = ash.secretManager.Region } + ash.awsMetadata.AwsRegion = awsRegion podIdentity := ash.secretManager.PodIdentity if podIdentity == nil { podIdentity = &kedav1alpha1.AuthPodIdentity{} @@ -100,7 +101,7 @@ func (ash *AwsSecretManagerHandler) Initialize(ctx context.Context, client clien return fmt.Errorf("pod identity provider %s not supported", podIdentity.Provider) } - config, err := awsutils.GetAwsConfig(ctx, awsRegion, ash.awsMetadata) + config, err := awsutils.GetAwsConfig(ctx, ash.awsMetadata) if err != nil { logger.Error(err, "Error getting credentials") return err From c8be1c7cd0a3bf08d7fc4c70f57df80be8cb7f94 Mon Sep 17 00:00:00 2001 From: Eng Zer Jun Date: Wed, 4 Dec 2024 18:34:21 +0800 Subject: [PATCH 15/15] refactor: replace experimental `maps` and `slices` with stdlib (#6372) Signed-off-by: Eng Zer Jun Signed-off-by: Jan Wozniak Co-authored-by: Jan Wozniak --- CHANGELOG.md | 1 + apis/eventing/v1alpha1/cloudeventsource_webhook.go | 2 +- go.mod | 6 ++---- pkg/eventemitter/eventfilter.go | 2 +- pkg/scalers/scalersconfig/typed_config.go | 11 ++++------- .../opentelemetry_metrics_test.go | 2 +- 6 files changed, 10 insertions(+), 14 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 266aea0115b..f28f04d14e5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -94,6 +94,7 @@ New deprecation(s): ### Other - **General**: Bump newrelic-client-go deps to 2.51.2 (latest) ([#6325](https://github.com/kedacore/keda/pull/6325)) +- **General**: refactor: replace experimental `maps` and `slices` with stdlib ([#6372](https://github.com/kedacore/keda/pull/6372)) ## v2.16.0 diff --git a/apis/eventing/v1alpha1/cloudeventsource_webhook.go b/apis/eventing/v1alpha1/cloudeventsource_webhook.go index af7e69d7fb4..2dc577665de 100644 --- a/apis/eventing/v1alpha1/cloudeventsource_webhook.go +++ b/apis/eventing/v1alpha1/cloudeventsource_webhook.go @@ -19,8 +19,8 @@ package v1alpha1 import ( "encoding/json" "fmt" + "slices" - "golang.org/x/exp/slices" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/runtime" ctrl "sigs.k8s.io/controller-runtime" diff --git a/go.mod b/go.mod index 1d90fed97f9..ab53e4324fa 100644 --- a/go.mod +++ b/go.mod @@ -1,8 +1,6 @@ module github.com/kedacore/keda/v2 -go 1.22.1 - -toolchain go1.23.3 +go 1.23.3 require ( cloud.google.com/go/compute/metadata v0.5.2 @@ -347,7 +345,7 @@ require ( go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect golang.org/x/crypto v0.28.0 - golang.org/x/exp v0.0.0-20240808152545-0cdaa3abc0fa + golang.org/x/exp v0.0.0-20240808152545-0cdaa3abc0fa // indirect golang.org/x/mod v0.21.0 // indirect golang.org/x/net v0.30.0 // indirect golang.org/x/sys v0.26.0 // indirect diff --git a/pkg/eventemitter/eventfilter.go b/pkg/eventemitter/eventfilter.go index 11af3af1ac8..b6b0dc32b0e 100644 --- a/pkg/eventemitter/eventfilter.go +++ b/pkg/eventemitter/eventfilter.go @@ -17,7 +17,7 @@ limitations under the License. package eventemitter import ( - "golang.org/x/exp/slices" + "slices" eventingv1alpha1 "github.com/kedacore/keda/v2/apis/eventing/v1alpha1" ) diff --git a/pkg/scalers/scalersconfig/typed_config.go b/pkg/scalers/scalersconfig/typed_config.go index 028028c8de0..62e5a71b9af 100644 --- a/pkg/scalers/scalersconfig/typed_config.go +++ b/pkg/scalers/scalersconfig/typed_config.go @@ -20,14 +20,13 @@ import ( "encoding/json" "errors" "fmt" + "maps" "net/url" "reflect" "runtime/debug" + "slices" "strconv" "strings" - - "golang.org/x/exp/maps" - "golang.org/x/exp/slices" ) // CustomValidator is an interface that can be implemented to validate the configuration of the typed config @@ -204,8 +203,7 @@ func (sc *ScalerConfig) setValue(field reflect.Value, params Params) error { } if !exists && !(params.Optional || params.IsDeprecated()) { if len(params.Order) == 0 { - apo := maps.Keys(allowedParsingOrderMap) - slices.Sort(apo) + apo := slices.Sorted(maps.Keys(allowedParsingOrderMap)) return fmt.Errorf("missing required parameter %q, no 'order' tag, provide any from %v", params.Name(), apo) } return fmt.Errorf("missing required parameter %q in %v", params.Name(), params.Order) @@ -463,8 +461,7 @@ func paramsFromTag(tag string, field reflect.StructField) (Params, error) { for _, po := range order { poTyped := ParsingOrder(strings.TrimSpace(po)) if !allowedParsingOrderMap[poTyped] { - apo := maps.Keys(allowedParsingOrderMap) - slices.Sort(apo) + apo := slices.Sorted(maps.Keys(allowedParsingOrderMap)) return params, fmt.Errorf("unknown parsing order value %s, has to be one of %s", po, apo) } params.Order = append(params.Order, poTyped) diff --git a/tests/sequential/opentelemetry_metrics/opentelemetry_metrics_test.go b/tests/sequential/opentelemetry_metrics/opentelemetry_metrics_test.go index be9fafdc46e..98e287e2d08 100644 --- a/tests/sequential/opentelemetry_metrics/opentelemetry_metrics_test.go +++ b/tests/sequential/opentelemetry_metrics/opentelemetry_metrics_test.go @@ -9,6 +9,7 @@ import ( "fmt" "maps" "os/exec" + "slices" "strings" "testing" "time" @@ -17,7 +18,6 @@ import ( "github.com/prometheus/common/expfmt" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "golang.org/x/exp/slices" corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/client-go/kubernetes"