From 85003278fd324177f6f0a2d3171d18c723759e8f Mon Sep 17 00:00:00 2001 From: joshvanl Date: Sun, 1 Sep 2024 04:17:47 +0100 Subject: [PATCH 01/27] Scheduler: Adds Kubernetes namespace controller When in Kubernetes mode, the Scheduler will delete Jobs of all kinds (currently cron and actor reminders) when the namespace they belong to is deleted. This prevents both squatting memory in Etcd, and more importantly, consuming resources scheduling jobs which have no consumers to signal. The Scheduler Namespace Controller will Delete all Jobs in a namespace when the Kubernetes Namespace is deleted. The controller uses leader election, meaning only one instance of the Scheduler will be running the controller at any one time. Uses an updated go-etcd-cron library version to use the new `DeletePrefixes` API method. The Scheduler has been given the permissions to get, list and watch namespaces in Kubernetes so the controller can function. The controller is only executed when the Scheduler is in Kubernetes mode (`--mode=kubernetes`). A new `--kubeconfig` CLI flag has been added to the Scheduler, which is used in integration testing so that it points to the mock Kubernetes API server. This isn't useful for end users so hasn't been documented or exposed in the Helm chart. Since the Namespace controller needs to call the Etcd Cron library, the Scheduler server has been refactored to move functionality into `/internal` of the server package, so that they can be shared between both the server and controller. Signed-off-by: joshvanl --- .../charts/dapr_rbac/templates/scheduler.yaml | 7 +- cmd/scheduler/app/app.go | 1 + cmd/scheduler/options/options.go | 10 + cmd/scheduler/options/options_test.go | 57 ++++ go.mod | 2 + go.sum | 2 + pkg/operator/api/authz/authz.go | 3 +- pkg/scheduler/server/api.go | 144 ++------- pkg/scheduler/server/config_test.go | 279 ++++++++++++++++++ .../server/{ => internal}/authz/authz.go | 2 +- .../server/{ => internal}/authz/authz_test.go | 4 +- .../server/internal/controller/controller.go | 110 +++++++ .../server/internal/controller/namespace.go | 56 ++++ .../internal/controller/namespace_test.go | 98 ++++++ pkg/scheduler/server/internal/cron/cron.go | 184 ++++++++++++ .../server/internal/cron/fake/fake.go | 55 ++++ .../server/internal/cron/fake/fake_test.go | 24 ++ .../server/internal/{ => pool}/conn.go | 2 +- .../server/internal/{ => pool}/pool.go | 4 +- .../server/internal/serialize/names.go | 21 ++ .../server/internal/serialize/serialize.go | 116 ++++++++ pkg/scheduler/server/server.go | 82 +++-- pkg/scheduler/server/server_test.go | 211 ------------- .../framework/manifest/manifest.go | 36 +++ .../framework/process/daprd/daprd.go | 8 + .../process/kubernetes/informer/informer.go | 12 +- .../framework/process/kubernetes/options.go | 9 + .../process/kubernetes/store/store.go | 11 + .../framework/process/placement/options.go | 16 +- .../framework/process/scheduler/options.go | 14 + .../framework/process/scheduler/scheduler.go | 13 +- .../suite/actors/reminders/scheduler/basic.go | 9 +- tests/integration/suite/daprd/jobs/jobs.go | 1 + .../suite/daprd/jobs/kubernetes/namespace.go | 152 ++++++++++ .../suite/scheduler/kubernetes/namespace.go | 125 ++++++++ .../integration/suite/scheduler/scheduler.go | 1 + 36 files changed, 1509 insertions(+), 372 deletions(-) create mode 100644 cmd/scheduler/options/options_test.go create mode 100644 pkg/scheduler/server/config_test.go rename pkg/scheduler/server/{ => internal}/authz/authz.go (95%) rename pkg/scheduler/server/{ => internal}/authz/authz_test.go (98%) create mode 100644 pkg/scheduler/server/internal/controller/controller.go create mode 100644 pkg/scheduler/server/internal/controller/namespace.go create mode 100644 pkg/scheduler/server/internal/controller/namespace_test.go create mode 100644 pkg/scheduler/server/internal/cron/cron.go create mode 100644 pkg/scheduler/server/internal/cron/fake/fake.go create mode 100644 pkg/scheduler/server/internal/cron/fake/fake_test.go rename pkg/scheduler/server/internal/{ => pool}/conn.go (99%) rename pkg/scheduler/server/internal/{ => pool}/pool.go (99%) create mode 100644 pkg/scheduler/server/internal/serialize/names.go create mode 100644 pkg/scheduler/server/internal/serialize/serialize.go create mode 100644 tests/integration/framework/manifest/manifest.go create mode 100644 tests/integration/suite/daprd/jobs/kubernetes/namespace.go create mode 100644 tests/integration/suite/scheduler/kubernetes/namespace.go diff --git a/charts/dapr/charts/dapr_rbac/templates/scheduler.yaml b/charts/dapr/charts/dapr_rbac/templates/scheduler.yaml index 2d43a4631db..72edfef3800 100644 --- a/charts/dapr/charts/dapr_rbac/templates/scheduler.yaml +++ b/charts/dapr/charts/dapr_rbac/templates/scheduler.yaml @@ -21,7 +21,10 @@ metadata: {{- range $key, $value := .Values.global.k8sLabels }} {{ $key }}: {{ tpl $value $ }} {{- end }} -rules: [] +rules: + - apiGroups: [""] + resources: ["namespaces"] + verbs: ["get", "list", "watch"] --- {{- if eq .Values.global.rbac.namespaced true }} kind: RoleBinding @@ -47,4 +50,4 @@ roleRef: kind: ClusterRole {{- end }} name: dapr-scheduler -{{- end }} \ No newline at end of file +{{- end }} diff --git a/cmd/scheduler/app/app.go b/cmd/scheduler/app/app.go index 2f1975c0d78..122550e0ac8 100644 --- a/cmd/scheduler/app/app.go +++ b/cmd/scheduler/app/app.go @@ -102,6 +102,7 @@ func Run() { DataDir: opts.EtcdDataDir, ReplicaCount: opts.ReplicaCount, ReplicaID: opts.ReplicaID, + KubeConfig: opts.KubeConfig, EtcdID: opts.ID, EtcdInitialPeers: opts.EtcdInitialPeers, EtcdClientPorts: opts.EtcdClientPorts, diff --git a/cmd/scheduler/options/options.go b/cmd/scheduler/options/options.go index e308279b5f6..8837ffd7fa1 100644 --- a/cmd/scheduler/options/options.go +++ b/cmd/scheduler/options/options.go @@ -42,6 +42,7 @@ type Options struct { SentryAddress string PlacementAddress string Mode string + KubeConfig *string ID string ReplicaID uint32 @@ -61,6 +62,7 @@ type Options struct { Metrics *metrics.FlagOptions taFile string + kubeconfig string etcdSpaceQuota string } @@ -83,6 +85,7 @@ func New(origArgs []string) (*Options, error) { fs.StringVar(&opts.taFile, "trust-anchors-file", securityConsts.ControlPlaneDefaultTrustAnchorsPath, "Filepath to the trust anchors for the Dapr control plane") fs.StringVar(&opts.SentryAddress, "sentry-address", fmt.Sprintf("dapr-sentry.%s.svc:443", security.CurrentNamespace()), "Address of the Sentry service") fs.StringVar(&opts.Mode, "mode", string(modes.StandaloneMode), "Runtime mode for Dapr Scheduler") + fs.StringVar(&opts.kubeconfig, "kubeconfig", "", "Kubernetes mode only. Absolute path to the kubeconfig file.") fs.StringVar(&opts.ID, "id", "dapr-scheduler-server-0", "Scheduler server ID") fs.Uint32Var(&opts.ReplicaCount, "replica-count", 1, "The total number of scheduler replicas in the cluster") @@ -134,5 +137,12 @@ func New(origArgs []string) (*Options, error) { log.Warnf("--etcd-space-quota of %s may be too low for production use. Consider increasing the value to 16Gi or larger.", etcdSpaceQuota.String()) } + if fs.Changed("kubeconfig") { + if opts.Mode != string(modes.KubernetesMode) { + return nil, fmt.Errorf("kubeconfig flag is only valid in --mode=kubernetes") + } + opts.KubeConfig = &opts.kubeconfig + } + return &opts, nil } diff --git a/cmd/scheduler/options/options_test.go b/cmd/scheduler/options/options_test.go new file mode 100644 index 00000000000..b17479afb16 --- /dev/null +++ b/cmd/scheduler/options/options_test.go @@ -0,0 +1,57 @@ +/* +Copyright 2024 The Dapr Authors +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package options + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestNew(t *testing.T) { + t.Run("don't error if only using defaults", func(t *testing.T) { + _, err := New([]string{}) + require.NoError(t, err) + }) + + t.Run("error when kubeconfig is set and using defaults", func(t *testing.T) { + _, err := New([]string{ + "--kubeconfig=" + t.TempDir(), + }) + require.Error(t, err) + }) + + t.Run("don't error when mode set to kubernetes", func(t *testing.T) { + _, err := New([]string{ + "--mode=kubernetes", + }) + require.NoError(t, err) + }) + + t.Run("error when mode set to standalone and kubeconfig is set", func(t *testing.T) { + _, err := New([]string{ + "--mode=standalone", + "--kubeconfig=" + t.TempDir(), + }) + require.Error(t, err) + }) + + t.Run("don't error when mode set to kubernetes and kubeconfig is set", func(t *testing.T) { + _, err := New([]string{ + "--mode=kubernetes", + "--kubeconfig=" + t.TempDir(), + }) + require.NoError(t, err) + }) +} diff --git a/go.mod b/go.mod index 61ff61a94a6..924c119302e 100644 --- a/go.mod +++ b/go.mod @@ -492,6 +492,8 @@ replace ( // Needed due to a deprecated method used in functional tests replace github.com/stretchr/testify => github.com/stretchr/testify v1.8.4 +replace github.com/diagridio/go-etcd-cron => github.com/joshvanl/go-etcd-cron v0.0.0-20240901031816-5173b9308d3e + // Uncomment for local development for testing with changes in the components-contrib && kit repositories. // Don't commit with this uncommented! // diff --git a/go.sum b/go.sum index c7533ce959e..4822f0bb339 100644 --- a/go.sum +++ b/go.sum @@ -1042,6 +1042,8 @@ github.com/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST github.com/jonboulle/clockwork v0.4.0/go.mod h1:xgRqUGwRcjKCO1vbZUEtSLrqKoPSsUpK7fnezOII0kc= github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY= github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= +github.com/joshvanl/go-etcd-cron v0.0.0-20240901031816-5173b9308d3e h1:sStHT4YuOotqyj5behRYAPGM9QdYXy1EiZXEmai0Nf4= +github.com/joshvanl/go-etcd-cron v0.0.0-20240901031816-5173b9308d3e/go.mod h1:yYzzG6/Qgq4jlRi/ZIri2zgKkxhghxiA8u7a4Q7tIYQ= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= diff --git a/pkg/operator/api/authz/authz.go b/pkg/operator/api/authz/authz.go index 610dfdac3f5..203773f78c5 100644 --- a/pkg/operator/api/authz/authz.go +++ b/pkg/operator/api/authz/authz.go @@ -15,6 +15,7 @@ package authz import ( "context" + "fmt" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -31,7 +32,7 @@ func Request(ctx context.Context, namespace string) (*spiffe.Parsed, error) { } if len(namespace) == 0 || id.Namespace() != namespace { - return nil, status.New(codes.PermissionDenied, "identity does not match requested namespace").Err() + return nil, status.New(codes.PermissionDenied, fmt.Sprintf("identity does not match requested namespace exp=%s got=%s", namespace, id.Namespace())).Err() } return id, nil diff --git a/pkg/scheduler/server/api.go b/pkg/scheduler/server/api.go index 857a27195ea..d86f9cd75c5 100644 --- a/pkg/scheduler/server/api.go +++ b/pkg/scheduler/server/api.go @@ -18,47 +18,37 @@ import ( "errors" "fmt" "strings" - "time" "github.com/diagridio/go-etcd-cron/api" - "google.golang.org/protobuf/types/known/anypb" + "google.golang.org/appengine/log" schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" "github.com/dapr/dapr/pkg/scheduler/monitoring" - "github.com/dapr/dapr/pkg/scheduler/server/internal" ) func (s *Server) ScheduleJob(ctx context.Context, req *schedulerv1pb.ScheduleJobRequest) (*schedulerv1pb.ScheduleJobResponse, error) { - select { - case <-ctx.Done(): - return nil, ctx.Err() - case <-s.readyCh: - } - - if err := s.authz.Metadata(ctx, req.GetMetadata()); err != nil { - return nil, err - } - - jobName, err := buildJobName(req.GetName(), req.GetMetadata()) + cron, err := s.cron.Client(ctx) if err != nil { return nil, err } - meta, err := anypb.New(req.GetMetadata()) + serialized, err := s.serializer.FromRequest(ctx, req) if err != nil { return nil, err } - job := &api.Job{ - Schedule: req.GetJob().Schedule, //nolint:protogetter - DueTime: req.GetJob().DueTime, //nolint:protogetter - Ttl: req.GetJob().Ttl, //nolint:protogetter - Repeats: req.GetJob().Repeats, //nolint:protogetter - Metadata: meta, - Payload: req.GetJob().GetData(), + job := req.GetJob() + + apiJob := &api.Job{ + Schedule: job.Schedule, //nolint:protogetter + DueTime: job.DueTime, //nolint:protogetter + Ttl: job.Ttl, //nolint:protogetter + Repeats: job.Repeats, //nolint:protogetter + Metadata: serialized.Metadata(), + Payload: job.GetData(), } - err = s.cron.Add(ctx, jobName, job) + err = cron.Add(ctx, serialized.Name(), apiJob) if err != nil { log.Errorf("error scheduling job %s: %s", req.GetName(), err) return nil, err @@ -68,24 +58,19 @@ func (s *Server) ScheduleJob(ctx context.Context, req *schedulerv1pb.ScheduleJob } func (s *Server) DeleteJob(ctx context.Context, req *schedulerv1pb.DeleteJobRequest) (*schedulerv1pb.DeleteJobResponse, error) { - select { - case <-ctx.Done(): - return nil, ctx.Err() - case <-s.readyCh: - } - - if err := s.authz.Metadata(ctx, req.GetMetadata()); err != nil { + cron, err := s.cron.Client(ctx) + if err != nil { return nil, err } - jobName, err := buildJobName(req.GetName(), req.GetMetadata()) + job, err := s.serializer.FromRequest(ctx, req) if err != nil { return nil, err } - err = s.cron.Delete(ctx, jobName) + err = cron.Delete(ctx, job.Name()) if err != nil { - log.Errorf("error deleting job %s: %s", jobName, err) + log.Errorf("error deleting job %s: %s", job.Name(), err) return nil, err } @@ -93,24 +78,19 @@ func (s *Server) DeleteJob(ctx context.Context, req *schedulerv1pb.DeleteJobRequ } func (s *Server) GetJob(ctx context.Context, req *schedulerv1pb.GetJobRequest) (*schedulerv1pb.GetJobResponse, error) { - select { - case <-ctx.Done(): - return nil, ctx.Err() - case <-s.readyCh: - } - - if err := s.authz.Metadata(ctx, req.GetMetadata()); err != nil { + cron, err := s.cron.Client(ctx) + if err != nil { return nil, err } - jobName, err := buildJobName(req.GetName(), req.GetMetadata()) + serialized, err := s.serializer.FromRequest(ctx, req) if err != nil { return nil, err } - job, err := s.cron.Get(ctx, jobName) + job, err := cron.Get(ctx, serialized.Name()) if err != nil { - log.Errorf("error getting job %s: %s", jobName, err) + log.Errorf("error getting job %s: %s", serialized.Name(), err) return nil, err } @@ -173,20 +153,12 @@ func (s *Server) ListJobs(ctx context.Context, req *schedulerv1pb.ListJobsReques // WatchJobs sends jobs to Dapr sidecars upon component changes. func (s *Server) WatchJobs(stream schedulerv1pb.Scheduler_WatchJobsServer) error { - req, err := stream.Recv() + initial, err := s.serializer.FromWatch(stream) if err != nil { return err } - if req.GetInitial() == nil { - return errors.New("initial request is required on stream connection") - } - - if err := s.authz.Initial(stream.Context(), req.GetInitial()); err != nil { - return err - } - - s.connectionPool.Add(req.GetInitial(), stream) + s.connectionPool.Add(initial, stream) monitoring.RecordSidecarsConnectedCount(1) defer monitoring.RecordSidecarsConnectedCount(-1) @@ -197,69 +169,3 @@ func (s *Server) WatchJobs(stream schedulerv1pb.Scheduler_WatchJobsServer) error return stream.Context().Err() } } - -func (s *Server) triggerJob(ctx context.Context, req *api.TriggerRequest) bool { - log.Debugf("Triggering job: %s", req.GetName()) - - ctx, cancel := context.WithTimeout(ctx, time.Second*45) - defer cancel() - - var meta schedulerv1pb.JobMetadata - if err := req.GetMetadata().UnmarshalTo(&meta); err != nil { - log.Errorf("Error unmarshalling metadata: %s", err) - return true - } - - idx := strings.LastIndex(req.GetName(), "||") - if idx == -1 || len(req.GetName()) <= idx+2 { - log.Errorf("Job name is malformed: %s", req.GetName()) - return true - } - - now := time.Now() - if err := s.connectionPool.Send(ctx, &internal.JobEvent{ - Name: req.GetName()[idx+2:], - Data: req.GetPayload(), - Metadata: &meta, - }); err != nil { - // TODO: add job to a queue or something to try later this should be - // another long running go routine that accepts this job on a channel - log.Errorf("Error sending job to connection stream: %s", err) - } - monitoring.RecordTriggerDuration(now) - - monitoring.RecordJobsTriggeredCount(&meta) - return true -} - -func buildJobName(name string, meta *schedulerv1pb.JobMetadata) (string, error) { - switch t := meta.GetTarget(); t.GetType().(type) { - case *schedulerv1pb.JobTargetMetadata_Actor: - actor := t.GetActor() - return joinStrings("actorreminder", meta.GetNamespace(), actor.GetType(), actor.GetId(), name), nil - case *schedulerv1pb.JobTargetMetadata_Job: - return joinStrings("app", meta.GetNamespace(), meta.GetAppId(), name), nil - default: - return "", fmt.Errorf("unknown job type: %v", t) - } -} - -func buildJobPrefix(meta *schedulerv1pb.JobMetadata) (string, error) { - switch t := meta.GetTarget(); t.GetType().(type) { - case *schedulerv1pb.JobTargetMetadata_Actor: - actor := t.GetActor() - s := joinStrings("actorreminder", meta.GetNamespace(), actor.GetType()) - if len(actor.GetId()) > 0 { - s = joinStrings(s, actor.GetId()) - } - return s, nil - case *schedulerv1pb.JobTargetMetadata_Job: - return joinStrings("app", meta.GetNamespace(), meta.GetAppId()), nil - default: - return "", fmt.Errorf("unknown job type: %v", t) - } -} - -func joinStrings(ss ...string) string { - return strings.Join(ss, "||") -} diff --git a/pkg/scheduler/server/config_test.go b/pkg/scheduler/server/config_test.go new file mode 100644 index 00000000000..6e67f7f7fb9 --- /dev/null +++ b/pkg/scheduler/server/config_test.go @@ -0,0 +1,279 @@ +/* +Copyright 2024 The Dapr 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 server + +import ( + "net/url" + "runtime" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/dapr/dapr/pkg/healthz" + "github.com/dapr/dapr/pkg/modes" +) + +func TestServerConf(t *testing.T) { + t.Run("KubernetesMode", func(t *testing.T) { + config, err := config(Options{ + Security: nil, + ListenAddress: "", + Port: 0, + Mode: modes.KubernetesMode, + ReplicaCount: 0, + ReplicaID: 0, + DataDir: "", + EtcdID: "id2", + EtcdInitialPeers: []string{"id1=http://localhost:5001", "id2=http://localhost:5002"}, + EtcdClientPorts: []string{"id1=5001", "id2=5002"}, + EtcdClientHTTPPorts: []string{"id1=5003", "id2=5004"}, + EtcdSpaceQuota: 0, + EtcdCompactionMode: "", + EtcdCompactionRetention: "", + Healthz: healthz.New(), + }) + require.NoError(t, err) + + assert.Equal(t, "id1=http://localhost:5001,id2=http://localhost:5002", config.InitialCluster) + + clientURL := url.URL{ + Scheme: "http", + Host: "0.0.0.0:5002", + } + + assert.Equal(t, clientURL, config.ListenPeerUrls[0]) + assert.Equal(t, clientURL, config.ListenClientUrls[0]) + + clientHTTPURL := url.URL{ + Scheme: "http", + Host: "0.0.0.0:5004", + } + assert.Equal(t, clientHTTPURL, config.ListenClientHttpUrls[0]) + }) + + t.Run("StandaloneMode", func(t *testing.T) { + config, err := config(Options{ + Security: nil, + ListenAddress: "", + Port: 0, + Mode: modes.StandaloneMode, + ReplicaCount: 0, + ReplicaID: 0, + DataDir: "./data", + EtcdID: "id2", + EtcdInitialPeers: []string{"id1=http://localhost:5001", "id2=http://localhost:5002"}, + EtcdClientPorts: []string{"id1=5001", "id2=5002"}, + EtcdClientHTTPPorts: []string{"id1=5003", "id2=5004"}, + EtcdSpaceQuota: 0, + EtcdCompactionMode: "", + EtcdCompactionRetention: "", + Healthz: healthz.New(), + }) + require.NoError(t, err) + + assert.Equal(t, "id1=http://localhost:5001,id2=http://localhost:5002", config.InitialCluster) + if runtime.GOOS == "windows" { + assert.Equal(t, "data\\default-id2", config.Dir) + } else { + assert.Equal(t, "data/default-id2", config.Dir) + } + + clientURL := url.URL{ + Scheme: "http", + Host: "localhost:5002", + } + + assert.Equal(t, clientURL, config.ListenPeerUrls[0]) + assert.Equal(t, clientURL, config.ListenClientUrls[0]) + + clientHTTPURL := url.URL{ + Scheme: "http", + Host: "localhost:5004", + } + assert.Equal(t, clientHTTPURL, config.ListenClientHttpUrls[0]) + }) + + t.Run("StandaloneMode without client http ports", func(t *testing.T) { + config, err := config(Options{ + Security: nil, + ListenAddress: "", + Port: 0, + Mode: modes.StandaloneMode, + ReplicaCount: 0, + ReplicaID: 0, + DataDir: "./data", + EtcdID: "id2", + EtcdInitialPeers: []string{"id1=http://localhost:5001", "id2=http://localhost:5002"}, + EtcdClientPorts: []string{"id1=5001", "id2=5002"}, + EtcdClientHTTPPorts: nil, + EtcdSpaceQuota: 0, + EtcdCompactionMode: "", + EtcdCompactionRetention: "", + Healthz: healthz.New(), + }) + require.NoError(t, err) + + assert.Equal(t, "id1=http://localhost:5001,id2=http://localhost:5002", config.InitialCluster) + + clientURL := url.URL{ + Scheme: "http", + Host: "localhost:5002", + } + + assert.Equal(t, clientURL, config.ListenPeerUrls[0]) + assert.Equal(t, clientURL, config.ListenClientUrls[0]) + + assert.Empty(t, config.ListenClientHttpUrls) + }) + + t.Run("StandaloneMode listen on 0.0.0.0 when a host", func(t *testing.T) { + config, err := config(Options{ + Security: nil, + ListenAddress: "", + Port: 0, + Mode: modes.StandaloneMode, + ReplicaCount: 0, + ReplicaID: 0, + DataDir: "./data", + EtcdID: "id2", + EtcdInitialPeers: []string{"id1=http://hello1:5001", "id2=http://hello2:5002"}, + EtcdClientPorts: []string{"id1=5001", "id2=5002"}, + EtcdClientHTTPPorts: nil, + EtcdSpaceQuota: 0, + EtcdCompactionMode: "", + EtcdCompactionRetention: "", + Healthz: healthz.New(), + }) + require.NoError(t, err) + + assert.Equal(t, "id1=http://hello1:5001,id2=http://hello2:5002", config.InitialCluster) + + clientURL := url.URL{ + Scheme: "http", + Host: "0.0.0.0:5002", + } + assert.Equal(t, clientURL, config.ListenPeerUrls[0]) + assert.Equal(t, clientURL, config.ListenClientUrls[0]) + assert.Empty(t, config.ListenClientHttpUrls) + }) + + t.Run("StandaloneMode listen on IP when an IP", func(t *testing.T) { + config, err := config(Options{ + Security: nil, + ListenAddress: "", + Port: 0, + Mode: modes.StandaloneMode, + ReplicaCount: 0, + ReplicaID: 0, + DataDir: "./data", + EtcdID: "id2", + EtcdInitialPeers: []string{"id1=http://1.2.3.4:5001", "id2=http://1.2.3.4:5002"}, + EtcdClientPorts: []string{"id1=5001", "id2=5002"}, + EtcdClientHTTPPorts: nil, + EtcdSpaceQuota: 0, + EtcdCompactionMode: "", + EtcdCompactionRetention: "", + Healthz: healthz.New(), + }) + require.NoError(t, err) + + assert.Equal(t, "id1=http://1.2.3.4:5001,id2=http://1.2.3.4:5002", config.InitialCluster) + + clientURL := url.URL{ + Scheme: "http", + Host: "1.2.3.4:5002", + } + assert.Equal(t, clientURL, config.ListenPeerUrls[0]) + assert.Equal(t, clientURL, config.ListenClientUrls[0]) + assert.Empty(t, config.ListenClientHttpUrls) + }) + + t.Run("StandaloneMode listen on HTTP IP when an IP", func(t *testing.T) { + config, err := config(Options{ + Security: nil, + ListenAddress: "", + Port: 0, + Mode: modes.StandaloneMode, + ReplicaCount: 0, + ReplicaID: 0, + DataDir: "./data", + EtcdID: "id2", + EtcdInitialPeers: []string{"id1=http://1.2.3.4:5001", "id2=http://1.2.3.4:5002"}, + EtcdClientPorts: []string{"id1=5001", "id2=5002"}, + EtcdClientHTTPPorts: []string{"id1=5003", "id2=5004"}, + EtcdSpaceQuota: 0, + EtcdCompactionMode: "", + EtcdCompactionRetention: "", + Healthz: healthz.New(), + }) + require.NoError(t, err) + + assert.Equal(t, "id1=http://1.2.3.4:5001,id2=http://1.2.3.4:5002", config.InitialCluster) + + clientURL := url.URL{ + Scheme: "http", + Host: "1.2.3.4:5002", + } + assert.Equal(t, clientURL, config.ListenPeerUrls[0]) + assert.Equal(t, clientURL, config.ListenClientUrls[0]) + httpURL, err := url.Parse("http://1.2.3.4:5004") + require.NoError(t, err) + assert.Equal(t, []url.URL{*httpURL}, config.ListenClientHttpUrls) + }) + + t.Run("expect error when giving bad client ports", func(t *testing.T) { + config, err := config(Options{ + Security: nil, + ListenAddress: "", + Port: 0, + Mode: modes.StandaloneMode, + ReplicaCount: 0, + ReplicaID: 0, + DataDir: "./data", + EtcdID: "id2", + EtcdInitialPeers: []string{"id1=http://1.2.3.4:5001", "id2=http://1.2.3.4:5002"}, + EtcdClientPorts: []string{"id15001"}, + EtcdClientHTTPPorts: nil, + EtcdSpaceQuota: 0, + EtcdCompactionMode: "", + EtcdCompactionRetention: "", + Healthz: healthz.New(), + }) + require.Error(t, err) + assert.Nil(t, config) + }) + + t.Run("expect error when giving bad HTTP client ports", func(t *testing.T) { + config, err := config(Options{ + Security: nil, + ListenAddress: "", + Port: 0, + Mode: modes.StandaloneMode, + ReplicaCount: 0, + ReplicaID: 0, + DataDir: "./data", + EtcdID: "id2", + EtcdInitialPeers: []string{"id1=http://1.2.3.4:5001", "id2=http://1.2.3.4:5002"}, + EtcdClientPorts: []string{"id=5001"}, + EtcdClientHTTPPorts: []string{"id15001"}, + EtcdSpaceQuota: 0, + EtcdCompactionMode: "", + EtcdCompactionRetention: "", + Healthz: healthz.New(), + }) + require.Error(t, err) + assert.Nil(t, config) + }) +} diff --git a/pkg/scheduler/server/authz/authz.go b/pkg/scheduler/server/internal/authz/authz.go similarity index 95% rename from pkg/scheduler/server/authz/authz.go rename to pkg/scheduler/server/internal/authz/authz.go index c0e9ec3b0ab..00e87f41b88 100644 --- a/pkg/scheduler/server/authz/authz.go +++ b/pkg/scheduler/server/internal/authz/authz.go @@ -43,7 +43,7 @@ func (a *Authz) Metadata(ctx context.Context, meta *schedulerv1pb.JobMetadata) e return a.authz(ctx, meta.GetNamespace(), meta.GetAppId()) } -func (a *Authz) Initial(ctx context.Context, initial *schedulerv1pb.WatchJobsRequestInitial) error { +func (a *Authz) WatchInitial(ctx context.Context, initial *schedulerv1pb.WatchJobsRequestInitial) error { return a.authz(ctx, initial.GetNamespace(), initial.GetAppId()) } diff --git a/pkg/scheduler/server/authz/authz_test.go b/pkg/scheduler/server/internal/authz/authz_test.go similarity index 98% rename from pkg/scheduler/server/authz/authz_test.go rename to pkg/scheduler/server/internal/authz/authz_test.go index cc09f757909..16c6643b7dc 100644 --- a/pkg/scheduler/server/authz/authz_test.go +++ b/pkg/scheduler/server/internal/authz/authz_test.go @@ -184,14 +184,14 @@ func Test_Initial(t *testing.T) { for name, test := range tests { t.Run(name, func(t *testing.T) { a := New(Options{fake.New().WithMTLSEnabled(true)}) - err := a.Initial(test.ctx, test.initial) + err := a.WatchInitial(test.ctx, test.initial) assert.Equal(t, test.expCode != nil, err != nil, "%v %v", test.expCode, err) if test.expCode != nil { assert.Equal(t, *test.expCode, status.Code(err)) } a = New(Options{fake.New().WithMTLSEnabled(false)}) - err = a.Initial(test.ctx, test.initial) + err = a.WatchInitial(test.ctx, test.initial) assert.Equal(t, test.nonMTlSCode != nil, err != nil, "%v %v", test.nonMTlSCode, err) if test.nonMTlSCode != nil { assert.Equal(t, *test.nonMTlSCode, status.Code(err)) diff --git a/pkg/scheduler/server/internal/controller/controller.go b/pkg/scheduler/server/internal/controller/controller.go new file mode 100644 index 00000000000..ada5b3b7ebf --- /dev/null +++ b/pkg/scheduler/server/internal/controller/controller.go @@ -0,0 +1,110 @@ +/* +Copyright 2024 The Dapr 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 controller + +import ( + "context" + "fmt" + "os" + + "github.com/go-logr/logr" + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/client-go/rest" + "k8s.io/client-go/tools/clientcmd" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/builder" + metricsserver "sigs.k8s.io/controller-runtime/pkg/metrics/server" + + "github.com/dapr/dapr/pkg/healthz" + "github.com/dapr/dapr/pkg/scheduler/server/internal/cron" + "github.com/dapr/kit/concurrency" + "github.com/dapr/kit/logger" +) + +var log = logger.NewLogger("dapr.scheduler.server.controller") + +type Options struct { + KubeConfig *string + Cron cron.Interface + Healthz healthz.Healthz +} + +func New(opts Options) (concurrency.Runner, error) { + scheme := runtime.NewScheme() + if err := corev1.AddToScheme(scheme); err != nil { + return nil, err + } + + var restConfig *rest.Config + var err error + if opts.KubeConfig != nil { + var kcf []byte + kcf, err = os.ReadFile(*opts.KubeConfig) + if err != nil { + return nil, fmt.Errorf("unable to read kubeconfig: %w", err) + } + + restConfig, err = clientcmd.RESTConfigFromKubeConfig(kcf) + if err != nil { + return nil, fmt.Errorf("unable to create rest config from kubeconfig %q: %w", *opts.KubeConfig, err) + } + } else { + restConfig, err = ctrl.GetConfig() + if err != nil { + return nil, err + } + } + + mgr, err := ctrl.NewManager(restConfig, ctrl.Options{ + Logger: logr.Discard(), + Scheme: scheme, + HealthProbeBindAddress: "0", + Metrics: metricsserver.Options{BindAddress: "0"}, + LeaderElectionID: "scheduler.dapr.io", + LeaderElectionReleaseOnCancel: true, + }) + if err != nil { + return nil, fmt.Errorf("unable to start manager: %w", err) + } + + if err := ctrl.NewControllerManagedBy(mgr). + Named("namespaces"). + For(new(corev1.Namespace), builder.OnlyMetadata). + Complete(&namespace{ + nsReader: mgr.GetCache(), + cron: opts.Cron, + }); err != nil { + return nil, fmt.Errorf("unable to complete controller: %w", err) + } + + hzTarget := opts.Healthz.AddTarget() + + return concurrency.NewRunnerManager( + mgr.Start, + func(ctx context.Context) error { + _, err := mgr.GetCache().GetInformer(ctx, new(corev1.Namespace)) + if err != nil { + return fmt.Errorf("unable to get informer: %w", err) + } + if !mgr.GetCache().WaitForCacheSync(ctx) { + return fmt.Errorf("unable to sync cache") + } + hzTarget.Ready() + log.Info("Controller ready") + <-ctx.Done() + return nil + }, + ).Run, nil +} diff --git a/pkg/scheduler/server/internal/controller/namespace.go b/pkg/scheduler/server/internal/controller/namespace.go new file mode 100644 index 00000000000..a66eaf6ed2a --- /dev/null +++ b/pkg/scheduler/server/internal/controller/namespace.go @@ -0,0 +1,56 @@ +/* +Copyright 2024 The Dapr 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 controller + +import ( + "context" + + corev1 "k8s.io/api/core/v1" + apierrors "k8s.io/apimachinery/pkg/api/errors" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/dapr/dapr/pkg/scheduler/server/internal/cron" + "github.com/dapr/dapr/pkg/scheduler/server/internal/serialize" +) + +type namespace struct { + cron cron.Interface + nsReader client.Reader +} + +func (n *namespace) Reconcile(ctx context.Context, req ctrl.Request) (ctrl.Result, error) { + log.Debugf("Reconciling namespace %s", req.Name) + + cronClient, err := n.cron.Client(ctx) + if err != nil { + log.Errorf("Failed to get etcd cron client: %s", err) + return ctrl.Result{}, err + } + + var ns corev1.Namespace + err = n.nsReader.Get(ctx, client.ObjectKey{Name: req.Name}, &ns) + if !apierrors.IsNotFound(err) { + return ctrl.Result{}, err + } + + log.Infof("Deleting jobs from deleted namespace %s", req.Name) + err = cronClient.DeletePrefixes(ctx, serialize.PrefixesFromNamespace(req.Name)...) + if err != nil { + log.Errorf("Failed to delete cron jobs for namespace %s: %s", req.Name, err) + return ctrl.Result{}, err + } + + return ctrl.Result{}, nil +} diff --git a/pkg/scheduler/server/internal/controller/namespace_test.go b/pkg/scheduler/server/internal/controller/namespace_test.go new file mode 100644 index 00000000000..a0a3106a2f4 --- /dev/null +++ b/pkg/scheduler/server/internal/controller/namespace_test.go @@ -0,0 +1,98 @@ +/* +Copyright 2024 The Dapr 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 controller + +import ( + "context" + "testing" + + etcdcron "github.com/diagridio/go-etcd-cron" + etcdcronfake "github.com/diagridio/go-etcd-cron/fake" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/apimachinery/pkg/types" + ctrl "sigs.k8s.io/controller-runtime" + clientfake "sigs.k8s.io/controller-runtime/pkg/client/fake" + + cronfake "github.com/dapr/dapr/pkg/scheduler/server/internal/cron/fake" +) + +func Test_Reconcile(t *testing.T) { + tests := map[string]struct { + ns *corev1.Namespace + cronClientError error + deletePError error + expErr error + expDeletePrefixes []string + }{ + "etcd cron client returns error, should return error": { + cronClientError: assert.AnError, + ns: &corev1.Namespace{ObjectMeta: metav1.ObjectMeta{Name: "test-ns"}}, + expErr: assert.AnError, + }, + "namespace exists, do nothing": { + ns: &corev1.Namespace{ObjectMeta: metav1.ObjectMeta{Name: "test-ns"}}, + }, + "namespace doesn't exist, should delete prefixes": { + expDeletePrefixes: []string{"actorreminder||test-ns", "app||test-ns"}, + }, + "error on deleting prefix should error": { + deletePError: assert.AnError, + expDeletePrefixes: []string{"actorreminder||test-ns", "app||test-ns"}, + expErr: assert.AnError, + }, + "different namespace, should still delete prefixes": { + ns: &corev1.Namespace{ObjectMeta: metav1.ObjectMeta{Name: "different-ns"}}, + expDeletePrefixes: []string{"actorreminder||test-ns", "app||test-ns"}, + }, + } + + req := ctrl.Request{NamespacedName: types.NamespacedName{Name: "test-ns"}} + for name, test := range tests { + test := test + t.Run(name, func(t *testing.T) { + t.Parallel() + + var prefixCalled []string + cron := cronfake.New().WithClient(func(context.Context) (etcdcron.Interface, error) { + etcdcron := etcdcronfake.New().WithDeletePrefixes(func(_ context.Context, prefixes ...string) error { + prefixCalled = append(prefixCalled, prefixes...) + return test.deletePError + }) + return etcdcron, test.cronClientError + }) + + scheme := runtime.NewScheme() + require.NoError(t, corev1.AddToScheme(scheme)) + + nsReaderBuilder := clientfake.NewClientBuilder().WithScheme(scheme) + if test.ns != nil { + nsReaderBuilder.WithObjects(test.ns) + } + + nsctrl := &namespace{ + cron: cron, + nsReader: nsReaderBuilder.Build(), + } + + result, err := nsctrl.Reconcile(context.Background(), req) + assert.Equal(t, ctrl.Result{}, result) + assert.Equal(t, test.expErr, err) + assert.ElementsMatch(t, test.expDeletePrefixes, prefixCalled) + }) + } +} diff --git a/pkg/scheduler/server/internal/cron/cron.go b/pkg/scheduler/server/internal/cron/cron.go new file mode 100644 index 00000000000..8226d67e9f1 --- /dev/null +++ b/pkg/scheduler/server/internal/cron/cron.go @@ -0,0 +1,184 @@ +/* +Copyright 2024 The Dapr 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 cron + +import ( + "context" + "errors" + "fmt" + "strings" + "time" + + etcdcron "github.com/diagridio/go-etcd-cron" + "github.com/diagridio/go-etcd-cron/api" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + + "github.com/dapr/dapr/pkg/healthz" + schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" + "github.com/dapr/dapr/pkg/scheduler/server/internal/pool" + "github.com/dapr/kit/concurrency" + "github.com/dapr/kit/logger" +) + +var log = logger.NewLogger("dapr.scheduler.server.cron") + +type Options struct { + ReplicaCount uint32 + ReplicaID uint32 + Config *embed.Config + ConnectionPool *pool.Pool + Healthz healthz.Healthz +} + +// Interface manages the cron framework, exposing a client to schedule jobs. +type Interface interface { + // Run starts the cron server, blocking until the context is canceled. + Run(ctx context.Context) error + + // Client returns a client to schedule jobs with the underlying cron + // framework and database. Blocks until Etcd and the Cron library are ready. + Client(ctx context.Context) (etcdcron.Interface, error) +} + +type cron struct { + replicaCount uint32 + replicaID uint32 + + config *embed.Config + connectionPool *pool.Pool + etcdcron etcdcron.Interface + + readyCh chan struct{} + hzETCD healthz.Target +} + +func New(opts Options) Interface { + return &cron{ + replicaCount: opts.ReplicaCount, + replicaID: opts.ReplicaID, + config: opts.Config, + connectionPool: opts.ConnectionPool, + readyCh: make(chan struct{}), + hzETCD: opts.Healthz.AddTarget(), + } +} + +func (c *cron) Run(ctx context.Context) error { + defer c.hzETCD.NotReady() + + log.Info("Starting etcd") + + etcd, err := embed.StartEtcd(c.config) + if err != nil { + return err + } + defer etcd.Close() + + select { + case <-etcd.Server.ReadyNotify(): + log.Info("Etcd server is ready!") + case <-ctx.Done(): + return ctx.Err() + } + + log.Info("Starting EtcdCron") + + endpoints := make([]string, 0, len(etcd.Clients)) + for _, peer := range etcd.Clients { + endpoints = append(endpoints, peer.Addr().String()) + } + + client, err := clientv3.New(clientv3.Config{ + Endpoints: endpoints, + }) + if err != nil { + return err + } + + // pass in initial cluster endpoints, but with client ports + c.etcdcron, err = etcdcron.New(etcdcron.Options{ + Client: client, + Namespace: "dapr", + PartitionID: c.replicaID, + PartitionTotal: c.replicaCount, + TriggerFn: c.triggerJob, + }) + if err != nil { + return fmt.Errorf("fail to create etcd-cron: %s", err) + } + close(c.readyCh) + + c.hzETCD.Ready() + + return concurrency.NewRunnerManager( + func(ctx context.Context) error { + if err := c.etcdcron.Run(ctx); !errors.Is(err, context.DeadlineExceeded) { + return err + } + return nil + }, + func(ctx context.Context) error { + defer log.Info("EtcdCron shutting down") + select { + case err := <-etcd.Err(): + return err + case <-ctx.Done(): + return nil + } + }, + ).Run(ctx) +} + +// Client returns the Cron client, blocking until Etcd and the Cron library are +// ready. +func (c *cron) Client(ctx context.Context) (etcdcron.Interface, error) { + select { + case <-c.readyCh: + return c.etcdcron, nil + case <-ctx.Done(): + return nil, ctx.Err() + } +} + +func (c *cron) triggerJob(ctx context.Context, req *api.TriggerRequest) bool { + log.Debugf("Triggering job: %s", req.GetName()) + + ctx, cancel := context.WithTimeout(ctx, time.Second*45) + defer cancel() + + var meta schedulerv1pb.JobMetadata + if err := req.GetMetadata().UnmarshalTo(&meta); err != nil { + log.Errorf("Error unmarshalling metadata: %s", err) + return true + } + + idx := strings.LastIndex(req.GetName(), "||") + if idx == -1 || len(req.GetName()) <= idx+2 { + log.Errorf("Job name is malformed: %s", req.GetName()) + return true + } + + if err := c.connectionPool.Send(ctx, &pool.JobEvent{ + Name: req.GetName()[idx+2:], + Data: req.GetPayload(), + Metadata: &meta, + }); err != nil { + // TODO: add job to a queue or something to try later this should be + // another long running go routine that accepts this job on a channel + log.Errorf("Error sending job to connection stream: %s", err) + } + + return true +} diff --git a/pkg/scheduler/server/internal/cron/fake/fake.go b/pkg/scheduler/server/internal/cron/fake/fake.go new file mode 100644 index 00000000000..6704a201c9d --- /dev/null +++ b/pkg/scheduler/server/internal/cron/fake/fake.go @@ -0,0 +1,55 @@ +/* +Copyright 2024 The Dapr 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 fake + +import ( + "context" + + etcdcron "github.com/diagridio/go-etcd-cron" +) + +type Fake struct { + runFn func(context.Context) error + clientFn func(context.Context) (etcdcron.Interface, error) +} + +func New() *Fake { + return &Fake{ + runFn: func(ctx context.Context) error { + <-ctx.Done() + return ctx.Err() + }, + clientFn: func(context.Context) (etcdcron.Interface, error) { + return nil, nil + }, + } +} + +func (f *Fake) WithRun(fn func(context.Context) error) *Fake { + f.runFn = fn + return f +} + +func (f *Fake) WithClient(fn func(context.Context) (etcdcron.Interface, error)) *Fake { + f.clientFn = fn + return f +} + +func (f *Fake) Run(ctx context.Context) error { + return f.runFn(ctx) +} + +func (f *Fake) Client(ctx context.Context) (etcdcron.Interface, error) { + return f.clientFn(ctx) +} diff --git a/pkg/scheduler/server/internal/cron/fake/fake_test.go b/pkg/scheduler/server/internal/cron/fake/fake_test.go new file mode 100644 index 00000000000..0af043d20d6 --- /dev/null +++ b/pkg/scheduler/server/internal/cron/fake/fake_test.go @@ -0,0 +1,24 @@ +/* +Copyright 2024 The Dapr 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 fake + +import ( + "testing" + + "github.com/dapr/dapr/pkg/scheduler/server/internal/cron" +) + +func Test_Fake(t *testing.T) { + var _ cron.Interface = new(Fake) +} diff --git a/pkg/scheduler/server/internal/conn.go b/pkg/scheduler/server/internal/pool/conn.go similarity index 99% rename from pkg/scheduler/server/internal/conn.go rename to pkg/scheduler/server/internal/pool/conn.go index 450c69d647e..b5dbe80d53a 100644 --- a/pkg/scheduler/server/internal/conn.go +++ b/pkg/scheduler/server/internal/pool/conn.go @@ -11,7 +11,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package internal +package pool import ( "context" diff --git a/pkg/scheduler/server/internal/pool.go b/pkg/scheduler/server/internal/pool/pool.go similarity index 99% rename from pkg/scheduler/server/internal/pool.go rename to pkg/scheduler/server/internal/pool/pool.go index e9efd4e6a76..f7d90bfc40e 100644 --- a/pkg/scheduler/server/internal/pool.go +++ b/pkg/scheduler/server/internal/pool/pool.go @@ -11,7 +11,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package internal +package pool import ( "context" @@ -52,7 +52,7 @@ type JobEvent struct { Metadata *schedulerv1pb.JobMetadata } -func NewPool() *Pool { +func New() *Pool { return &Pool{ nsPool: make(map[string]*namespacedPool), closeCh: make(chan struct{}), diff --git a/pkg/scheduler/server/internal/serialize/names.go b/pkg/scheduler/server/internal/serialize/names.go new file mode 100644 index 00000000000..8bece16e960 --- /dev/null +++ b/pkg/scheduler/server/internal/serialize/names.go @@ -0,0 +1,21 @@ +/* +Copyright 2024 The Dapr 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 serialize + +func PrefixesFromNamespace(namespace string) []string { + return []string{ + "actorreminder||" + namespace, + "app||" + namespace, + } +} diff --git a/pkg/scheduler/server/internal/serialize/serialize.go b/pkg/scheduler/server/internal/serialize/serialize.go new file mode 100644 index 00000000000..fd9495cd8ab --- /dev/null +++ b/pkg/scheduler/server/internal/serialize/serialize.go @@ -0,0 +1,116 @@ +/* +Copyright 2024 The Dapr 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 serialize + +import ( + "context" + "errors" + "fmt" + "strings" + + "google.golang.org/protobuf/types/known/anypb" + + schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" + "github.com/dapr/dapr/pkg/scheduler/server/internal/authz" + "github.com/dapr/dapr/pkg/security" +) + +type Options struct { + Security security.Handler +} + +type Serializer struct { + authz *authz.Authz +} + +type Request interface { + GetName() string + GetMetadata() *schedulerv1pb.JobMetadata +} + +type Job struct { + name string + meta *anypb.Any +} + +func New(opts Options) *Serializer { + return &Serializer{ + authz: authz.New(authz.Options{ + Security: opts.Security, + }), + } +} + +func (s *Serializer) FromRequest(ctx context.Context, req Request) (*Job, error) { + if err := s.authz.Metadata(ctx, req.GetMetadata()); err != nil { + return nil, err + } + + meta, err := anypb.New(req.GetMetadata()) + if err != nil { + return nil, err + } + + name, err := buildJobName(req.GetName(), req.GetMetadata()) + if err != nil { + return nil, err + } + + return &Job{ + meta: meta, + name: name, + }, nil +} + +func (s *Serializer) FromWatch(stream schedulerv1pb.Scheduler_WatchJobsServer) (*schedulerv1pb.WatchJobsRequestInitial, error) { + req, err := stream.Recv() + if err != nil { + return nil, err + } + + initial := req.GetInitial() + if initial == nil { + return nil, errors.New("initial request is required on stream connection") + } + + if err := s.authz.WatchInitial(stream.Context(), initial); err != nil { + return nil, err + } + + return initial, nil +} + +func (j *Job) Name() string { + return j.name +} + +func (j *Job) Metadata() *anypb.Any { + return j.meta +} + +func buildJobName(name string, meta *schedulerv1pb.JobMetadata) (string, error) { + joinStrings := func(ss ...string) string { + return strings.Join(ss, "||") + } + + switch t := meta.GetTarget(); t.GetType().(type) { + case *schedulerv1pb.JobTargetMetadata_Actor: + actor := t.GetActor() + return joinStrings("actorreminder", meta.GetNamespace(), actor.GetType(), actor.GetId(), name), nil + case *schedulerv1pb.JobTargetMetadata_Job: + return joinStrings("app", meta.GetNamespace(), meta.GetAppId(), name), nil + default: + return "", fmt.Errorf("unknown job type: %v", t) + } +} diff --git a/pkg/scheduler/server/server.go b/pkg/scheduler/server/server.go index dd95f3c9be0..9990db14bf4 100644 --- a/pkg/scheduler/server/server.go +++ b/pkg/scheduler/server/server.go @@ -22,17 +22,22 @@ import ( "sync" "sync/atomic" +<<<<<<< HEAD "github.com/diagridio/go-etcd-cron/api" "github.com/diagridio/go-etcd-cron/cron" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" +======= +>>>>>>> 4735e839d (Scheduler: Adds Kubernetes namespace controller) "google.golang.org/grpc" "github.com/dapr/dapr/pkg/healthz" "github.com/dapr/dapr/pkg/modes" schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" - "github.com/dapr/dapr/pkg/scheduler/server/authz" - "github.com/dapr/dapr/pkg/scheduler/server/internal" + "github.com/dapr/dapr/pkg/scheduler/server/internal/controller" + "github.com/dapr/dapr/pkg/scheduler/server/internal/cron" + "github.com/dapr/dapr/pkg/scheduler/server/internal/pool" + "github.com/dapr/dapr/pkg/scheduler/server/internal/serialize" "github.com/dapr/dapr/pkg/security" "github.com/dapr/kit/concurrency" "github.com/dapr/kit/logger" @@ -46,6 +51,7 @@ type Options struct { ListenAddress string Port int Mode modes.DaprMode + KubeConfig *string ReplicaCount uint32 ReplicaID uint32 DataDir string @@ -65,21 +71,24 @@ type Options struct { type Server struct { listenAddress string port int - replicaCount uint32 - replicaID uint32 sec security.Handler +<<<<<<< HEAD authz *authz.Authz config *embed.Config cron api.Interface connectionPool *internal.Pool // Connection pool for sidecars +======= + serializer *serialize.Serializer + cron cron.Interface + connectionPool *pool.Pool // Connection pool for sidecars + controller concurrency.Runner +>>>>>>> 4735e839d (Scheduler: Adds Kubernetes namespace controller) hzAPIServer healthz.Target - hzETCD healthz.Target wg sync.WaitGroup running atomic.Bool - readyCh chan struct{} closeCh chan struct{} } @@ -90,21 +99,41 @@ func New(opts Options) (*Server, error) { return nil, err } + connectionPool := pool.New() + + cron := cron.New(cron.Options{ + ReplicaCount: opts.ReplicaCount, + ReplicaID: opts.ReplicaID, + ConnectionPool: connectionPool, + Config: config, + Healthz: opts.Healthz, + }) + + var ctrl concurrency.Runner + if opts.Mode == modes.KubernetesMode { + var err error + ctrl, err = controller.New(controller.Options{ + KubeConfig: opts.KubeConfig, + Cron: cron, + Healthz: opts.Healthz, + }) + if err != nil { + return nil, err + } + } + return &Server{ - port: opts.Port, - listenAddress: opts.ListenAddress, - replicaCount: opts.ReplicaCount, - replicaID: opts.ReplicaID, - sec: opts.Security, - authz: authz.New(authz.Options{ + port: opts.Port, + listenAddress: opts.ListenAddress, + sec: opts.Security, + connectionPool: connectionPool, + controller: ctrl, + cron: cron, + serializer: serialize.New(serialize.Options{ Security: opts.Security, }), - config: config, - connectionPool: internal.NewPool(), - readyCh: make(chan struct{}), - closeCh: make(chan struct{}), - hzAPIServer: opts.Healthz.AddTarget(), - hzETCD: opts.Healthz.AddTarget(), + closeCh: make(chan struct{}), + hzAPIServer: opts.Healthz.AddTarget(), }, nil } @@ -115,17 +144,23 @@ func (s *Server) Run(ctx context.Context) error { log.Info("Dapr Scheduler is starting...") - defer s.wg.Wait() - return concurrency.NewRunnerManager( + runners := []concurrency.Runner{ s.connectionPool.Run, s.runServer, - s.runEtcdCron, + s.cron.Run, func(ctx context.Context) error { <-ctx.Done() close(s.closeCh) return nil }, - ).Run(ctx) + } + + if s.controller != nil { + runners = append(runners, s.controller) + } + + defer s.wg.Wait() + return concurrency.NewRunnerManager(runners...).Run(ctx) } func (s *Server) runServer(ctx context.Context) error { @@ -162,6 +197,7 @@ func (s *Server) runServer(ctx context.Context) error { }, ).Run(ctx) } +<<<<<<< HEAD func (s *Server) runEtcdCron(ctx context.Context) error { defer s.hzETCD.NotReady() @@ -229,3 +265,5 @@ func (s *Server) runEtcdCron(ctx context.Context) error { }, ).Run(ctx) } +======= +>>>>>>> 4735e839d (Scheduler: Adds Kubernetes namespace controller) diff --git a/pkg/scheduler/server/server_test.go b/pkg/scheduler/server/server_test.go index ffe7de18aef..4b1126f4008 100644 --- a/pkg/scheduler/server/server_test.go +++ b/pkg/scheduler/server/server_test.go @@ -14,15 +14,10 @@ limitations under the License. package server import ( - "net/url" - "runtime" "testing" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - - "github.com/dapr/dapr/pkg/healthz" - "github.com/dapr/dapr/pkg/modes" ) func TestParseClientPorts(t *testing.T) { @@ -65,209 +60,3 @@ func TestParseClientPorts(t *testing.T) { assert.Equal(t, "5001", clientPorts["scheduler1"]) }) } - -func TestServerConf(t *testing.T) { - t.Run("KubernetesMode", func(t *testing.T) { - opts := Options{ - Security: nil, - ListenAddress: "", - Port: 0, - Mode: modes.KubernetesMode, - ReplicaCount: 0, - ReplicaID: 0, - DataDir: "", - EtcdID: "id2", - EtcdInitialPeers: []string{"id1=http://localhost:5001", "id2=http://localhost:5002"}, - EtcdClientPorts: []string{"id1=5001", "id2=5002"}, - EtcdClientHTTPPorts: []string{"id1=5003", "id2=5004"}, - EtcdSpaceQuota: 0, - EtcdCompactionMode: "", - EtcdCompactionRetention: "", - Healthz: healthz.New(), - } - - s, err := New(opts) - if err != nil { - t.Fatalf("failed to create server: %s", err) - } - - config := s.config - - assert.Equal(t, "id1=http://localhost:5001,id2=http://localhost:5002", config.InitialCluster) - - clientURL := url.URL{ - Scheme: "http", - Host: "0.0.0.0:5002", - } - - assert.Equal(t, clientURL, config.ListenPeerUrls[0]) - assert.Equal(t, clientURL, config.ListenClientUrls[0]) - - clientHTTPURL := url.URL{ - Scheme: "http", - Host: "0.0.0.0:5004", - } - assert.Equal(t, clientHTTPURL, config.ListenClientHttpUrls[0]) - }) - - t.Run("StandaloneMode", func(t *testing.T) { - opts := Options{ - Security: nil, - ListenAddress: "", - Port: 0, - Mode: modes.StandaloneMode, - ReplicaCount: 0, - ReplicaID: 0, - DataDir: "./data", - EtcdID: "id2", - EtcdInitialPeers: []string{"id1=http://localhost:5001", "id2=http://localhost:5002"}, - EtcdClientPorts: []string{"id1=5001", "id2=5002"}, - EtcdClientHTTPPorts: []string{"id1=5003", "id2=5004"}, - EtcdSpaceQuota: 0, - EtcdCompactionMode: "", - EtcdCompactionRetention: "", - Healthz: healthz.New(), - } - - s, err := New(opts) - if err != nil { - t.Fatalf("failed to create server: %s", err) - } - - config := s.config - - assert.Equal(t, "id1=http://localhost:5001,id2=http://localhost:5002", config.InitialCluster) - if runtime.GOOS == "windows" { - assert.Equal(t, "data\\default-id2", config.Dir) - } else { - assert.Equal(t, "data/default-id2", config.Dir) - } - - clientURL := url.URL{ - Scheme: "http", - Host: "localhost:5002", - } - - assert.Equal(t, clientURL, config.ListenPeerUrls[0]) - assert.Equal(t, clientURL, config.ListenClientUrls[0]) - - clientHTTPURL := url.URL{ - Scheme: "http", - Host: "localhost:5004", - } - assert.Equal(t, clientHTTPURL, config.ListenClientHttpUrls[0]) - }) - - t.Run("StandaloneMode without client http ports", func(t *testing.T) { - opts := Options{ - Security: nil, - ListenAddress: "", - Port: 0, - Mode: modes.StandaloneMode, - ReplicaCount: 0, - ReplicaID: 0, - DataDir: "./data", - EtcdID: "id2", - EtcdInitialPeers: []string{"id1=http://localhost:5001", "id2=http://localhost:5002"}, - EtcdClientPorts: []string{"id1=5001", "id2=5002"}, - EtcdClientHTTPPorts: nil, - EtcdSpaceQuota: 0, - EtcdCompactionMode: "", - EtcdCompactionRetention: "", - Healthz: healthz.New(), - } - - s, err := New(opts) - if err != nil { - t.Fatalf("failed to create server: %s", err) - } - - config := s.config - - assert.Equal(t, "id1=http://localhost:5001,id2=http://localhost:5002", config.InitialCluster) - - clientURL := url.URL{ - Scheme: "http", - Host: "localhost:5002", - } - - assert.Equal(t, clientURL, config.ListenPeerUrls[0]) - assert.Equal(t, clientURL, config.ListenClientUrls[0]) - - assert.Empty(t, config.ListenClientHttpUrls) - }) - - t.Run("StandaloneMode listen on 0.0.0.0 when a host", func(t *testing.T) { - opts := Options{ - Security: nil, - ListenAddress: "", - Port: 0, - Mode: modes.StandaloneMode, - ReplicaCount: 0, - ReplicaID: 0, - DataDir: "./data", - EtcdID: "id2", - EtcdInitialPeers: []string{"id1=http://hello1:5001", "id2=http://hello2:5002"}, - EtcdClientPorts: []string{"id1=5001", "id2=5002"}, - EtcdClientHTTPPorts: nil, - EtcdSpaceQuota: 0, - EtcdCompactionMode: "", - EtcdCompactionRetention: "", - Healthz: healthz.New(), - } - - s, err := New(opts) - if err != nil { - t.Fatalf("failed to create server: %s", err) - } - - config := s.config - - assert.Equal(t, "id1=http://hello1:5001,id2=http://hello2:5002", config.InitialCluster) - - clientURL := url.URL{ - Scheme: "http", - Host: "0.0.0.0:5002", - } - assert.Equal(t, clientURL, config.ListenPeerUrls[0]) - assert.Equal(t, clientURL, config.ListenClientUrls[0]) - assert.Empty(t, config.ListenClientHttpUrls) - }) - - t.Run("StandaloneMode listen on IP when an IP", func(t *testing.T) { - opts := Options{ - Security: nil, - ListenAddress: "", - Port: 0, - Mode: modes.StandaloneMode, - ReplicaCount: 0, - ReplicaID: 0, - DataDir: "./data", - EtcdID: "id2", - EtcdInitialPeers: []string{"id1=http://1.2.3.4:5001", "id2=http://1.2.3.4:5002"}, - EtcdClientPorts: []string{"id1=5001", "id2=5002"}, - EtcdClientHTTPPorts: nil, - EtcdSpaceQuota: 0, - EtcdCompactionMode: "", - EtcdCompactionRetention: "", - Healthz: healthz.New(), - } - - s, err := New(opts) - if err != nil { - t.Fatalf("failed to create server: %s", err) - } - - config := s.config - - assert.Equal(t, "id1=http://1.2.3.4:5001,id2=http://1.2.3.4:5002", config.InitialCluster) - - clientURL := url.URL{ - Scheme: "http", - Host: "1.2.3.4:5002", - } - assert.Equal(t, clientURL, config.ListenPeerUrls[0]) - assert.Equal(t, clientURL, config.ListenClientUrls[0]) - assert.Empty(t, config.ListenClientHttpUrls) - }) -} diff --git a/tests/integration/framework/manifest/manifest.go b/tests/integration/framework/manifest/manifest.go new file mode 100644 index 00000000000..7fd202a76ff --- /dev/null +++ b/tests/integration/framework/manifest/manifest.go @@ -0,0 +1,36 @@ +/* +Copyright 2024 The Dapr 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 manifest + +import ( + apiextv1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + "github.com/dapr/dapr/pkg/apis/common" + compapi "github.com/dapr/dapr/pkg/apis/components/v1alpha1" +) + +func ActorInMemoryStateComponent(ns, name string) compapi.Component { + return compapi.Component{ + TypeMeta: metav1.TypeMeta{Kind: "Component", APIVersion: "dapr.io/v1alpha1"}, + ObjectMeta: metav1.ObjectMeta{Namespace: ns, Name: name}, + Spec: compapi.ComponentSpec{ + Type: "state.in-memory", + Version: "v1", + Metadata: []common.NameValuePair{ + {Name: "actorStateStore", Value: common.DynamicValue{JSON: apiextv1.JSON{Raw: []byte(`"true"`)}}}, + }, + }, + } +} diff --git a/tests/integration/framework/process/daprd/daprd.go b/tests/integration/framework/process/daprd/daprd.go index 4d7d9a60928..9b96f312936 100644 --- a/tests/integration/framework/process/daprd/daprd.go +++ b/tests/integration/framework/process/daprd/daprd.go @@ -473,3 +473,11 @@ func (d *Daprd) meta(t assert.TestingT, ctx context.Context) metaResponse { return meta } + +func (d *Daprd) ActorInvokeURL(actorType, actorID, method string) string { + return fmt.Sprintf("http://%s/v1.0/actors/%s/%s/method/%s", d.HTTPAddress(), actorType, actorID, method) +} + +func (d *Daprd) ActorReminderURL(actorType, actorID, method string) string { + return fmt.Sprintf("http://%s/v1.0/actors/%s/%s/reminders/%s", d.HTTPAddress(), actorType, actorID, method) +} diff --git a/tests/integration/framework/process/kubernetes/informer/informer.go b/tests/integration/framework/process/kubernetes/informer/informer.go index c563c870c46..f8d40f236b0 100644 --- a/tests/integration/framework/process/kubernetes/informer/informer.go +++ b/tests/integration/framework/process/kubernetes/informer/informer.go @@ -74,9 +74,15 @@ func (i *Informer) Handler(t *testing.T, wrapped http.Handler) http.HandlerFunc split = split[2:] } if split[0] == "namespaces" { - split = split[2:] + if len(split) > 1 { + split = split[2:] + gvk.Kind = split[0] + } else { + gvk.Kind = "Namespace" + } + } else { + gvk.Kind = split[0] } - gvk.Kind = split[0] w.Header().Add("Transfer-Encoding", "chunked") w.Header().Add("Content-Type", "application/json") @@ -147,6 +153,8 @@ func (i *Informer) objToGVK(t *testing.T, obj runtime.Object) schema.GroupVersio return schema.GroupVersionKind{Group: "", Version: "v1", Kind: "secrets"} case *corev1.ConfigMap: return schema.GroupVersionKind{Group: "", Version: "v1", Kind: "configmaps"} + case *corev1.Namespace: + return schema.GroupVersionKind{Group: "", Version: "v1", Kind: "namespaces"} default: require.Fail(t, "unknown type: %T", obj) return schema.GroupVersionKind{} diff --git a/tests/integration/framework/process/kubernetes/options.go b/tests/integration/framework/process/kubernetes/options.go index f9b4fa98c09..176b7fc41d6 100644 --- a/tests/integration/framework/process/kubernetes/options.go +++ b/tests/integration/framework/process/kubernetes/options.go @@ -86,6 +86,10 @@ func WithClusterDaprSubscriptionV2ListFromStore(t *testing.T, store *store.Store return handleClusterListResourceFromStore(t, "/apis/dapr.io/v2alpha1/subscriptions", store) } +func WithClusterNamespaceListFromStore(t *testing.T, store *store.Store) Option { + return handleClusterListResourceFromStore(t, "/api/v1/namespaces", store) +} + func WithClusterDaprHTTPEndpointList(t *testing.T, endpoints *httpendapi.HTTPEndpointList) Option { return handleClusterListResource(t, "/apis/dapr.io/v1alpha1/httpendpoints", endpoints) } @@ -110,6 +114,10 @@ func WithClusterServiceAccountList(t *testing.T, services *corev1.ServiceAccount return handleClusterListResource(t, "/api/v1/serviceaccounts", services) } +func WithClusterNamespaceList(t *testing.T, namespaces *corev1.NamespaceList) Option { + return handleClusterListResource(t, "/api/v1/namespaces", namespaces) +} + func WithDaprConfigurationGet(t *testing.T, config *configapi.Configuration) Option { return handleGetResource(t, "/apis/dapr.io/v1alpha1", "configurations", config.Namespace, config.Name, config) } @@ -155,6 +163,7 @@ func WithBaseOperatorAPI(t *testing.T, td spiffeid.TrustDomain, ns string, sentr WithClusterDaprSubscriptionListV2(t, &subv2api.SubscriptionList{TypeMeta: metav1.TypeMeta{APIVersion: "dapr.io/v2alpha1", Kind: "SubscriptionList"}}), WithClusterDaprHTTPEndpointList(t, &httpendapi.HTTPEndpointList{TypeMeta: metav1.TypeMeta{APIVersion: "dapr.io/v1alpha1", Kind: "HTTPEndpointList"}}), WithClusterDaprResiliencyList(t, &resapi.ResiliencyList{TypeMeta: metav1.TypeMeta{APIVersion: "dapr.io/v1alpha1", Kind: "ResiliencyList"}}), + WithClusterNamespaceList(t, &corev1.NamespaceList{TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "NamespaceList"}}), } { op(o) } diff --git a/tests/integration/framework/process/kubernetes/store/store.go b/tests/integration/framework/process/kubernetes/store/store.go index f6635520523..9d914187002 100644 --- a/tests/integration/framework/process/kubernetes/store/store.go +++ b/tests/integration/framework/process/kubernetes/store/store.go @@ -45,6 +45,17 @@ func (s *Store) Add(objs ...client.Object) { } } +func (s *Store) Delete(objs ...client.Object) { + s.lock.Lock() + defer s.lock.Unlock() + if s.objs == nil { + s.objs = make(map[string]client.Object) + } + for _, obj := range objs { + delete(s.objs, obj.GetNamespace()+"/"+obj.GetName()) + } +} + func (s *Store) Set(objs ...client.Object) { s.lock.Lock() defer s.lock.Unlock() diff --git a/tests/integration/framework/process/placement/options.go b/tests/integration/framework/process/placement/options.go index cb0f21ceef6..1ee7fc8ff78 100644 --- a/tests/integration/framework/process/placement/options.go +++ b/tests/integration/framework/process/placement/options.go @@ -13,7 +13,13 @@ limitations under the License. package placement -import "github.com/dapr/dapr/tests/integration/framework/process/exec" +import ( + "testing" + + "github.com/dapr/dapr/tests/integration/framework/process/exec" + "github.com/dapr/dapr/tests/integration/framework/process/sentry" + "github.com/dapr/kit/ptr" +) // Option is a function that configures the process. type Option func(*options) @@ -91,6 +97,14 @@ func WithSentryAddress(sentryAddress string) Option { } } +func WithSentry(t *testing.T, sentry *sentry.Sentry) Option { + return func(o *options) { + o.tlsEnabled = true + o.sentryAddress = ptr.Of(sentry.Address()) + o.trustAnchorsFile = ptr.Of(sentry.TrustAnchorsFile(t)) + } +} + func WithTrustAnchorsFile(file string) Option { return func(o *options) { o.trustAnchorsFile = &file diff --git a/tests/integration/framework/process/scheduler/options.go b/tests/integration/framework/process/scheduler/options.go index 2c2fabc0f78..184c48eb248 100644 --- a/tests/integration/framework/process/scheduler/options.go +++ b/tests/integration/framework/process/scheduler/options.go @@ -37,6 +37,8 @@ type options struct { listenAddress string sentry *sentry.Sentry dataDir *string + kubeconfig *string + mode *string } func WithExecOptions(execOptions ...exec.Option) Option { @@ -123,3 +125,15 @@ func WithDataDir(dataDir string) Option { o.dataDir = &dataDir } } + +func WithKubeconfig(kubeconfig string) Option { + return func(o *options) { + o.kubeconfig = &kubeconfig + } +} + +func WithMode(mode string) Option { + return func(o *options) { + o.mode = &mode + } +} diff --git a/tests/integration/framework/process/scheduler/scheduler.go b/tests/integration/framework/process/scheduler/scheduler.go index 7e4cea0ed7d..4a2008cfc6c 100644 --- a/tests/integration/framework/process/scheduler/scheduler.go +++ b/tests/integration/framework/process/scheduler/scheduler.go @@ -127,6 +127,13 @@ func New(t *testing.T, fopts ...Option) *Scheduler { ) } + if opts.kubeconfig != nil { + args = append(args, "--kubeconfig="+*opts.kubeconfig) + } + if opts.mode != nil { + args = append(args, "--mode="+*opts.mode) + } + clientPorts := make(map[string]string) for _, input := range opts.etcdClientPorts { idAndPort := strings.Split(input, "=") @@ -336,12 +343,12 @@ func (s *Scheduler) Metrics(t *testing.T, ctx context.Context) map[string]float6 return metrics } -func (s *Scheduler) EtcdClient(t *testing.T) *clientv3.Client { +func (s *Scheduler) ETCDClient(t *testing.T) *clientv3.Client { t.Helper() client, err := clientv3.New(clientv3.Config{ - Endpoints: []string{"127.0.0.1:" + s.EtcdClientPort()}, - DialTimeout: 15 * time.Second, + Endpoints: []string{fmt.Sprintf("127.0.0.1:%s", s.EtcdClientPort())}, + DialTimeout: 40 * time.Second, }) require.NoError(t, err) diff --git a/tests/integration/suite/actors/reminders/scheduler/basic.go b/tests/integration/suite/actors/reminders/scheduler/basic.go index e95ebc424f6..b3d3717d8c6 100644 --- a/tests/integration/suite/actors/reminders/scheduler/basic.go +++ b/tests/integration/suite/actors/reminders/scheduler/basic.go @@ -15,7 +15,6 @@ package scheduler import ( "context" - "fmt" "net/http" "os" "path/filepath" @@ -96,9 +95,8 @@ func (b *basic) Run(t *testing.T, ctx context.Context) { client := client.HTTP(t) - daprdURL := fmt.Sprintf("http://%s/v1.0/actors/myactortype/myactorid", b.daprd.HTTPAddress()) - - req, err := http.NewRequestWithContext(ctx, http.MethodPost, daprdURL+"/method/foo", nil) + aurl := b.daprd.ActorInvokeURL("myactortype", "myactorid", "foo") + req, err := http.NewRequestWithContext(ctx, http.MethodPost, aurl, nil) require.NoError(t, err) require.EventuallyWithT(t, func(c *assert.CollectT) { @@ -110,7 +108,8 @@ func (b *basic) Run(t *testing.T, ctx context.Context) { }, time.Second*10, time.Millisecond*10, "actor not ready in time") body := `{"dueTime": "1s", "data": "reminderdata"}` - req, err = http.NewRequestWithContext(ctx, http.MethodPost, daprdURL+"/reminders/remindermethod", strings.NewReader(body)) + aurl = b.daprd.ActorReminderURL("myactortype", "myactorid", "remindermethod") + req, err = http.NewRequestWithContext(ctx, http.MethodPost, aurl, strings.NewReader(body)) require.NoError(t, err) resp, err := client.Do(req) diff --git a/tests/integration/suite/daprd/jobs/jobs.go b/tests/integration/suite/daprd/jobs/jobs.go index dedf3a74775..a9c09733fe3 100644 --- a/tests/integration/suite/daprd/jobs/jobs.go +++ b/tests/integration/suite/daprd/jobs/jobs.go @@ -16,5 +16,6 @@ package jobs import ( _ "github.com/dapr/dapr/tests/integration/suite/daprd/jobs/grpc" _ "github.com/dapr/dapr/tests/integration/suite/daprd/jobs/http" + _ "github.com/dapr/dapr/tests/integration/suite/daprd/jobs/kubernetes" _ "github.com/dapr/dapr/tests/integration/suite/daprd/jobs/streaming" ) diff --git a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go new file mode 100644 index 00000000000..76663465afa --- /dev/null +++ b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go @@ -0,0 +1,152 @@ +/* +Copyright 2024 The Dapr 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 kubernetes + +import ( + "context" + "net/http" + "testing" + "time" + + "github.com/spiffe/go-spiffe/v2/spiffeid" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + compapi "github.com/dapr/dapr/pkg/apis/components/v1alpha1" + configapi "github.com/dapr/dapr/pkg/apis/configuration/v1alpha1" + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/manifest" + "github.com/dapr/dapr/tests/integration/framework/process/daprd" + "github.com/dapr/dapr/tests/integration/framework/process/http/app" + "github.com/dapr/dapr/tests/integration/framework/process/kubernetes" + "github.com/dapr/dapr/tests/integration/framework/process/kubernetes/store" + "github.com/dapr/dapr/tests/integration/framework/process/operator" + "github.com/dapr/dapr/tests/integration/framework/process/placement" + "github.com/dapr/dapr/tests/integration/framework/process/scheduler" + "github.com/dapr/dapr/tests/integration/framework/process/sentry" + "github.com/dapr/dapr/tests/integration/suite" + "github.com/dapr/kit/ptr" +) + +func init() { + suite.Register(new(namespace)) +} + +type namespace struct { + store *store.Store + daprd *daprd.Daprd + scheduler *scheduler.Scheduler +} + +func (n *namespace) Setup(t *testing.T) []framework.Option { + sentry := sentry.New(t) + + app := app.New(t, + app.WithConfig(`{"entities": ["myactortype"]}`), + app.WithHandlerFunc("/actors/myactortype/myactorid/method/foo", func(http.ResponseWriter, *http.Request) {}), + ) + + n.store = store.New(metav1.GroupVersionKind{ + Version: "v1", + Kind: "Namespace", + }) + n.store.Add(&corev1.Namespace{ + TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, + ObjectMeta: metav1.ObjectMeta{Name: "default"}, + }) + + kubeapi := kubernetes.New(t, + kubernetes.WithBaseOperatorAPI(t, spiffeid.RequireTrustDomainFromString("localhost"), "default", sentry.Port()), + kubernetes.WithClusterDaprConfigurationList(t, &configapi.ConfigurationList{ + Items: []configapi.Configuration{{ + ObjectMeta: metav1.ObjectMeta{Namespace: "default", Name: "schedulerreminders"}, + Spec: configapi.ConfigurationSpec{ + Features: []configapi.FeatureSpec{{Name: "SchedulerReminders", Enabled: ptr.Of(true)}}, + }, + }}, + }), + kubernetes.WithClusterDaprComponentList(t, &compapi.ComponentList{ + Items: []compapi.Component{manifest.ActorInMemoryStateComponent("default", "foo")}, + }), + kubernetes.WithClusterNamespaceListFromStore(t, n.store), + ) + + n.scheduler = scheduler.New(t, + scheduler.WithSentry(sentry), + scheduler.WithKubeconfig(kubeapi.KubeconfigPath(t)), + scheduler.WithMode("kubernetes"), + ) + + operator := operator.New(t, + operator.WithNamespace("default"), + operator.WithKubeconfigPath(kubeapi.KubeconfigPath(t)), + operator.WithTrustAnchorsFile(sentry.TrustAnchorsFile(t)), + ) + + placement := placement.New(t, placement.WithSentry(t, sentry)) + + n.daprd = daprd.New(t, + daprd.WithMode("kubernetes"), + daprd.WithNamespace("default"), + daprd.WithSentry(t, sentry), + daprd.WithSchedulerAddresses(n.scheduler.Address()), + daprd.WithAppPort(app.Port()), + daprd.WithDisableK8sSecretStore(true), + daprd.WithControlPlaneAddress(operator.Address()), + daprd.WithPlacementAddresses(placement.Address()), + daprd.WithConfigs("schedulerreminders"), + ) + + return []framework.Option{ + framework.WithProcesses(app, sentry, kubeapi, n.scheduler, placement, operator, n.daprd), + } +} + +func (n *namespace) Run(t *testing.T, ctx context.Context) { + n.daprd.WaitUntilRunning(t, ctx) + + client := n.daprd.GRPCClient(t, ctx) + _, err := client.ScheduleJobAlpha1(ctx, &rtv1.ScheduleJobRequest{ + Job: &rtv1.Job{Name: "test", Schedule: ptr.Of("@daily")}, + }) + require.NoError(t, err) + + _, err = client.RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + Name: "test", + DueTime: "1000s", + ActorType: "myactortype", + ActorId: "myactorid", + }) + require.NoError(t, err) + + etcdClient := n.scheduler.ETCDClient(t).KV + resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + require.NoError(t, err) + assert.Len(t, resp.Kvs, 2) + + n.store.Delete(&corev1.Namespace{ + TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, + ObjectMeta: metav1.ObjectMeta{Name: "default"}, + }) + + assert.EventuallyWithT(t, func(c *assert.CollectT) { + resp, err = etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + require.NoError(t, err) + assert.Empty(c, resp.Kvs) + }, time.Second*10, 10*time.Millisecond) +} diff --git a/tests/integration/suite/scheduler/kubernetes/namespace.go b/tests/integration/suite/scheduler/kubernetes/namespace.go new file mode 100644 index 00000000000..789ae6c5e00 --- /dev/null +++ b/tests/integration/suite/scheduler/kubernetes/namespace.go @@ -0,0 +1,125 @@ +/* +Copyright 2024 The Dapr 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 kubernetes + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/kubernetes" + "github.com/dapr/dapr/tests/integration/framework/process/kubernetes/store" + "github.com/dapr/dapr/tests/integration/framework/process/scheduler" + "github.com/dapr/dapr/tests/integration/framework/process/sentry" + "github.com/dapr/dapr/tests/integration/suite" + "github.com/dapr/kit/ptr" +) + +func init() { + suite.Register(new(namespace)) +} + +type namespace struct { + sentry *sentry.Sentry + scheduler *scheduler.Scheduler + store *store.Store +} + +func (n *namespace) Setup(t *testing.T) []framework.Option { + n.sentry = sentry.New(t) + + n.store = store.New(metav1.GroupVersionKind{ + Version: "v1", + Kind: "Namespace", + }) + n.store.Add(&corev1.Namespace{ + TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, + ObjectMeta: metav1.ObjectMeta{Name: "default"}, + }) + + kubeapi := kubernetes.New(t, + kubernetes.WithClusterNamespaceListFromStore(t, n.store), + ) + + n.scheduler = scheduler.New(t, + scheduler.WithSentry(n.sentry), + scheduler.WithKubeconfig(kubeapi.KubeconfigPath(t)), + scheduler.WithMode("kubernetes"), + ) + + return []framework.Option{ + framework.WithProcesses(n.sentry, kubeapi, n.scheduler), + } +} + +func (n *namespace) Run(t *testing.T, ctx context.Context) { + n.sentry.WaitUntilRunning(t, ctx) + n.scheduler.WaitUntilRunning(t, ctx) + + client := n.scheduler.ClientMTLS(t, ctx, "myapp") + + _, err := client.ScheduleJob(ctx, &schedulerv1pb.ScheduleJobRequest{ + Name: "testJob", + Job: &schedulerv1pb.Job{Schedule: ptr.Of("@daily")}, + Metadata: &schedulerv1pb.JobMetadata{ + AppId: "myapp", + Namespace: "default", + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Job{ + Job: new(schedulerv1pb.TargetJob), + }, + }, + }, + }) + require.NoError(t, err) + + _, err = client.ScheduleJob(ctx, &schedulerv1pb.ScheduleJobRequest{ + Name: "testJob", + Job: &schedulerv1pb.Job{Schedule: ptr.Of("@daily")}, + Metadata: &schedulerv1pb.JobMetadata{ + AppId: "myapp", + Namespace: "default", + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Actor{ + Actor: &schedulerv1pb.TargetActorReminder{Id: "test", Type: "test"}, + }, + }, + }, + }) + require.NoError(t, err) + + etcdClient := n.scheduler.ETCDClient(t).KV + resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + require.NoError(t, err) + assert.Len(t, resp.Kvs, 2) + + n.store.Delete(&corev1.Namespace{ + TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, + ObjectMeta: metav1.ObjectMeta{Name: "default"}, + }) + + assert.EventuallyWithT(t, func(c *assert.CollectT) { + resp, err = etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + require.NoError(t, err) + assert.Empty(c, resp.Kvs) + }, time.Second*10, 10*time.Millisecond) +} diff --git a/tests/integration/suite/scheduler/scheduler.go b/tests/integration/suite/scheduler/scheduler.go index a7899fdaf02..5e61c59e293 100644 --- a/tests/integration/suite/scheduler/scheduler.go +++ b/tests/integration/suite/scheduler/scheduler.go @@ -17,6 +17,7 @@ import ( _ "github.com/dapr/dapr/tests/integration/suite/scheduler/api" _ "github.com/dapr/dapr/tests/integration/suite/scheduler/authz" _ "github.com/dapr/dapr/tests/integration/suite/scheduler/helm" + _ "github.com/dapr/dapr/tests/integration/suite/scheduler/kubernetes" _ "github.com/dapr/dapr/tests/integration/suite/scheduler/metrics" _ "github.com/dapr/dapr/tests/integration/suite/scheduler/quorum" ) From ef21d2515367e437d07f04b8b3c0e4dd36d67f49 Mon Sep 17 00:00:00 2001 From: joshvanl Date: Sun, 8 Sep 2024 17:34:23 +0100 Subject: [PATCH 02/27] Fix namespace controller integration tests Signed-off-by: joshvanl --- .../server/internal/controller/controller.go | 3 +- .../process/kubernetes/informer/informer.go | 2 +- .../suite/daprd/jobs/kubernetes/namespace.go | 36 +++++++++---------- 3 files changed, 19 insertions(+), 22 deletions(-) diff --git a/pkg/scheduler/server/internal/controller/controller.go b/pkg/scheduler/server/internal/controller/controller.go index ada5b3b7ebf..85c9d8352d7 100644 --- a/pkg/scheduler/server/internal/controller/controller.go +++ b/pkg/scheduler/server/internal/controller/controller.go @@ -24,7 +24,6 @@ import ( "k8s.io/client-go/rest" "k8s.io/client-go/tools/clientcmd" ctrl "sigs.k8s.io/controller-runtime" - "sigs.k8s.io/controller-runtime/pkg/builder" metricsserver "sigs.k8s.io/controller-runtime/pkg/metrics/server" "github.com/dapr/dapr/pkg/healthz" @@ -81,7 +80,7 @@ func New(opts Options) (concurrency.Runner, error) { if err := ctrl.NewControllerManagedBy(mgr). Named("namespaces"). - For(new(corev1.Namespace), builder.OnlyMetadata). + For(new(corev1.Namespace)). Complete(&namespace{ nsReader: mgr.GetCache(), cron: opts.Cron, diff --git a/tests/integration/framework/process/kubernetes/informer/informer.go b/tests/integration/framework/process/kubernetes/informer/informer.go index f8d40f236b0..b6fb5c12807 100644 --- a/tests/integration/framework/process/kubernetes/informer/informer.go +++ b/tests/integration/framework/process/kubernetes/informer/informer.go @@ -78,7 +78,7 @@ func (i *Informer) Handler(t *testing.T, wrapped http.Handler) http.HandlerFunc split = split[2:] gvk.Kind = split[0] } else { - gvk.Kind = "Namespace" + gvk.Kind = "namespaces" } } else { gvk.Kind = split[0] diff --git a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go index 76663465afa..9f5f3925213 100644 --- a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go +++ b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go @@ -34,7 +34,6 @@ import ( "github.com/dapr/dapr/tests/integration/framework/process/daprd" "github.com/dapr/dapr/tests/integration/framework/process/http/app" "github.com/dapr/dapr/tests/integration/framework/process/kubernetes" - "github.com/dapr/dapr/tests/integration/framework/process/kubernetes/store" "github.com/dapr/dapr/tests/integration/framework/process/operator" "github.com/dapr/dapr/tests/integration/framework/process/placement" "github.com/dapr/dapr/tests/integration/framework/process/scheduler" @@ -48,9 +47,10 @@ func init() { } type namespace struct { - store *store.Store daprd *daprd.Daprd scheduler *scheduler.Scheduler + placement *placement.Placement + kubeapi *kubernetes.Kubernetes } func (n *namespace) Setup(t *testing.T) []framework.Option { @@ -61,16 +61,7 @@ func (n *namespace) Setup(t *testing.T) []framework.Option { app.WithHandlerFunc("/actors/myactortype/myactorid/method/foo", func(http.ResponseWriter, *http.Request) {}), ) - n.store = store.New(metav1.GroupVersionKind{ - Version: "v1", - Kind: "Namespace", - }) - n.store.Add(&corev1.Namespace{ - TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, - ObjectMeta: metav1.ObjectMeta{Name: "default"}, - }) - - kubeapi := kubernetes.New(t, + n.kubeapi = kubernetes.New(t, kubernetes.WithBaseOperatorAPI(t, spiffeid.RequireTrustDomainFromString("localhost"), "default", sentry.Port()), kubernetes.WithClusterDaprConfigurationList(t, &configapi.ConfigurationList{ Items: []configapi.Configuration{{ @@ -83,22 +74,27 @@ func (n *namespace) Setup(t *testing.T) []framework.Option { kubernetes.WithClusterDaprComponentList(t, &compapi.ComponentList{ Items: []compapi.Component{manifest.ActorInMemoryStateComponent("default", "foo")}, }), - kubernetes.WithClusterNamespaceListFromStore(t, n.store), + kubernetes.WithClusterNamespaceList(t, &corev1.NamespaceList{ + Items: []corev1.Namespace{{ + TypeMeta: metav1.TypeMeta{Kind: "Namespace", APIVersion: "v1"}, + ObjectMeta: metav1.ObjectMeta{Name: "default"}, + }}, + }), ) n.scheduler = scheduler.New(t, scheduler.WithSentry(sentry), - scheduler.WithKubeconfig(kubeapi.KubeconfigPath(t)), + scheduler.WithKubeconfig(n.kubeapi.KubeconfigPath(t)), scheduler.WithMode("kubernetes"), ) operator := operator.New(t, operator.WithNamespace("default"), - operator.WithKubeconfigPath(kubeapi.KubeconfigPath(t)), + operator.WithKubeconfigPath(n.kubeapi.KubeconfigPath(t)), operator.WithTrustAnchorsFile(sentry.TrustAnchorsFile(t)), ) - placement := placement.New(t, placement.WithSentry(t, sentry)) + n.placement = placement.New(t, placement.WithSentry(t, sentry)) n.daprd = daprd.New(t, daprd.WithMode("kubernetes"), @@ -108,17 +104,19 @@ func (n *namespace) Setup(t *testing.T) []framework.Option { daprd.WithAppPort(app.Port()), daprd.WithDisableK8sSecretStore(true), daprd.WithControlPlaneAddress(operator.Address()), - daprd.WithPlacementAddresses(placement.Address()), + daprd.WithPlacementAddresses(n.placement.Address()), daprd.WithConfigs("schedulerreminders"), ) return []framework.Option{ - framework.WithProcesses(app, sentry, kubeapi, n.scheduler, placement, operator, n.daprd), + framework.WithProcesses(app, sentry, n.kubeapi, n.scheduler, n.placement, operator, n.daprd), } } func (n *namespace) Run(t *testing.T, ctx context.Context) { n.daprd.WaitUntilRunning(t, ctx) + n.scheduler.WaitUntilRunning(t, ctx) + n.placement.WaitUntilRunning(t, ctx) client := n.daprd.GRPCClient(t, ctx) _, err := client.ScheduleJobAlpha1(ctx, &rtv1.ScheduleJobRequest{ @@ -139,7 +137,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { require.NoError(t, err) assert.Len(t, resp.Kvs, 2) - n.store.Delete(&corev1.Namespace{ + n.kubeapi.Informer().Delete(t, &corev1.Namespace{ TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, ObjectMeta: metav1.ObjectMeta{Name: "default"}, }) From ebf1c69dc86274ff2c2c00a56b245dd74b5cd85d Mon Sep 17 00:00:00 2001 From: joshvanl Date: Mon, 9 Sep 2024 22:48:22 +0100 Subject: [PATCH 03/27] Adds Eventually for getting KV for reminder registration Signed-off-by: joshvanl --- .../suite/daprd/jobs/kubernetes/namespace.go | 10 ++++++---- .../suite/scheduler/kubernetes/namespace.go | 10 ++++++---- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go index 9f5f3925213..09d4d168bcf 100644 --- a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go +++ b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go @@ -133,9 +133,11 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { require.NoError(t, err) etcdClient := n.scheduler.ETCDClient(t).KV - resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) - require.NoError(t, err) - assert.Len(t, resp.Kvs, 2) + assert.EventuallyWithT(t, func(c *assert.CollectT) { + resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + require.NoError(t, err) + assert.Len(c, resp.Kvs, 2) + }, time.Second*10, 10*time.Millisecond) n.kubeapi.Informer().Delete(t, &corev1.Namespace{ TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, @@ -143,7 +145,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { }) assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err = etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) require.NoError(t, err) assert.Empty(c, resp.Kvs) }, time.Second*10, 10*time.Millisecond) diff --git a/tests/integration/suite/scheduler/kubernetes/namespace.go b/tests/integration/suite/scheduler/kubernetes/namespace.go index 789ae6c5e00..2cdc9e985c3 100644 --- a/tests/integration/suite/scheduler/kubernetes/namespace.go +++ b/tests/integration/suite/scheduler/kubernetes/namespace.go @@ -108,9 +108,11 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { require.NoError(t, err) etcdClient := n.scheduler.ETCDClient(t).KV - resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) - require.NoError(t, err) - assert.Len(t, resp.Kvs, 2) + assert.EventuallyWithT(t, func(c *assert.CollectT) { + resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + require.NoError(t, err) + assert.Len(c, resp.Kvs, 2) + }, time.Second*10, 10*time.Millisecond) n.store.Delete(&corev1.Namespace{ TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, @@ -118,7 +120,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { }) assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err = etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) require.NoError(t, err) assert.Empty(c, resp.Kvs) }, time.Second*10, 10*time.Millisecond) From fd7457f4de0970aa37470c8bf81455a9254ba70d Mon Sep 17 00:00:00 2001 From: joshvanl Date: Tue, 10 Sep 2024 15:46:51 +0100 Subject: [PATCH 04/27] Update diagridio/go-etcd-cron to v0.3.0 Signed-off-by: joshvanl --- go.mod | 3 +- go.sum | 4 +- pkg/scheduler/server/api.go | 14 +--- .../internal/controller/namespace_test.go | 6 +- pkg/scheduler/server/internal/cron/cron.go | 8 +- .../server/internal/cron/fake/fake.go | 10 +-- .../server/internal/serialize/serialize.go | 28 ++++++- pkg/scheduler/server/server.go | 84 ------------------- .../framework/process/scheduler/scheduler.go | 2 +- 9 files changed, 44 insertions(+), 115 deletions(-) diff --git a/go.mod b/go.mod index 924c119302e..35274102a3d 100644 --- a/go.mod +++ b/go.mod @@ -68,6 +68,7 @@ require ( golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 golang.org/x/net v0.29.0 golang.org/x/sync v0.8.0 + google.golang.org/appengine v1.6.8 google.golang.org/genproto/googleapis/api v0.0.0-20240924160255-9d4c2d233b61 google.golang.org/genproto/googleapis/rpc v0.0.0-20240924160255-9d4c2d233b61 google.golang.org/grpc v1.67.0 @@ -492,8 +493,6 @@ replace ( // Needed due to a deprecated method used in functional tests replace github.com/stretchr/testify => github.com/stretchr/testify v1.8.4 -replace github.com/diagridio/go-etcd-cron => github.com/joshvanl/go-etcd-cron v0.0.0-20240901031816-5173b9308d3e - // Uncomment for local development for testing with changes in the components-contrib && kit repositories. // Don't commit with this uncommented! // diff --git a/go.sum b/go.sum index 4822f0bb339..6d0c5c7f2b5 100644 --- a/go.sum +++ b/go.sum @@ -1042,8 +1042,6 @@ github.com/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST github.com/jonboulle/clockwork v0.4.0/go.mod h1:xgRqUGwRcjKCO1vbZUEtSLrqKoPSsUpK7fnezOII0kc= github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY= github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= -github.com/joshvanl/go-etcd-cron v0.0.0-20240901031816-5173b9308d3e h1:sStHT4YuOotqyj5behRYAPGM9QdYXy1EiZXEmai0Nf4= -github.com/joshvanl/go-etcd-cron v0.0.0-20240901031816-5173b9308d3e/go.mod h1:yYzzG6/Qgq4jlRi/ZIri2zgKkxhghxiA8u7a4Q7tIYQ= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= @@ -2274,6 +2272,8 @@ google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= +google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20170818010345-ee236bd376b0/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= diff --git a/pkg/scheduler/server/api.go b/pkg/scheduler/server/api.go index d86f9cd75c5..a200208823d 100644 --- a/pkg/scheduler/server/api.go +++ b/pkg/scheduler/server/api.go @@ -20,7 +20,6 @@ import ( "strings" "github.com/diagridio/go-etcd-cron/api" - "google.golang.org/appengine/log" schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" "github.com/dapr/dapr/pkg/scheduler/monitoring" @@ -111,22 +110,17 @@ func (s *Server) GetJob(ctx context.Context, req *schedulerv1pb.GetJobRequest) ( } func (s *Server) ListJobs(ctx context.Context, req *schedulerv1pb.ListJobsRequest) (*schedulerv1pb.ListJobsResponse, error) { - select { - case <-ctx.Done(): - return nil, ctx.Err() - case <-s.readyCh: - } - - if err := s.authz.Metadata(ctx, req.GetMetadata()); err != nil { + cron, err := s.cron.Client(ctx) + if err != nil { return nil, err } - prefix, err := buildJobPrefix(req.GetMetadata()) + prefix, err := s.serializer.PrefixFromList(ctx, req.GetMetadata()) if err != nil { return nil, err } - list, err := s.cron.List(ctx, prefix) + list, err := cron.List(ctx, prefix) if err != nil { return nil, fmt.Errorf("failed to query job list: %w", err) } diff --git a/pkg/scheduler/server/internal/controller/namespace_test.go b/pkg/scheduler/server/internal/controller/namespace_test.go index a0a3106a2f4..51d5a899a30 100644 --- a/pkg/scheduler/server/internal/controller/namespace_test.go +++ b/pkg/scheduler/server/internal/controller/namespace_test.go @@ -17,8 +17,8 @@ import ( "context" "testing" - etcdcron "github.com/diagridio/go-etcd-cron" - etcdcronfake "github.com/diagridio/go-etcd-cron/fake" + "github.com/diagridio/go-etcd-cron/cron" + etcdcronfake "github.com/diagridio/go-etcd-cron/tests/fake" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" corev1 "k8s.io/api/core/v1" @@ -68,7 +68,7 @@ func Test_Reconcile(t *testing.T) { t.Parallel() var prefixCalled []string - cron := cronfake.New().WithClient(func(context.Context) (etcdcron.Interface, error) { + cron := cronfake.New().WithClient(func(context.Context) (cron.Interface, error) { etcdcron := etcdcronfake.New().WithDeletePrefixes(func(_ context.Context, prefixes ...string) error { prefixCalled = append(prefixCalled, prefixes...) return test.deletePError diff --git a/pkg/scheduler/server/internal/cron/cron.go b/pkg/scheduler/server/internal/cron/cron.go index 8226d67e9f1..71869813f66 100644 --- a/pkg/scheduler/server/internal/cron/cron.go +++ b/pkg/scheduler/server/internal/cron/cron.go @@ -20,8 +20,8 @@ import ( "strings" "time" - etcdcron "github.com/diagridio/go-etcd-cron" "github.com/diagridio/go-etcd-cron/api" + etcdcron "github.com/diagridio/go-etcd-cron/cron" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" @@ -49,7 +49,7 @@ type Interface interface { // Client returns a client to schedule jobs with the underlying cron // framework and database. Blocks until Etcd and the Cron library are ready. - Client(ctx context.Context) (etcdcron.Interface, error) + Client(ctx context.Context) (api.Interface, error) } type cron struct { @@ -58,7 +58,7 @@ type cron struct { config *embed.Config connectionPool *pool.Pool - etcdcron etcdcron.Interface + etcdcron api.Interface readyCh chan struct{} hzETCD healthz.Target @@ -143,7 +143,7 @@ func (c *cron) Run(ctx context.Context) error { // Client returns the Cron client, blocking until Etcd and the Cron library are // ready. -func (c *cron) Client(ctx context.Context) (etcdcron.Interface, error) { +func (c *cron) Client(ctx context.Context) (api.Interface, error) { select { case <-c.readyCh: return c.etcdcron, nil diff --git a/pkg/scheduler/server/internal/cron/fake/fake.go b/pkg/scheduler/server/internal/cron/fake/fake.go index 6704a201c9d..041acb12a69 100644 --- a/pkg/scheduler/server/internal/cron/fake/fake.go +++ b/pkg/scheduler/server/internal/cron/fake/fake.go @@ -16,12 +16,12 @@ package fake import ( "context" - etcdcron "github.com/diagridio/go-etcd-cron" + "github.com/diagridio/go-etcd-cron/api" ) type Fake struct { runFn func(context.Context) error - clientFn func(context.Context) (etcdcron.Interface, error) + clientFn func(context.Context) (api.Interface, error) } func New() *Fake { @@ -30,7 +30,7 @@ func New() *Fake { <-ctx.Done() return ctx.Err() }, - clientFn: func(context.Context) (etcdcron.Interface, error) { + clientFn: func(context.Context) (api.Interface, error) { return nil, nil }, } @@ -41,7 +41,7 @@ func (f *Fake) WithRun(fn func(context.Context) error) *Fake { return f } -func (f *Fake) WithClient(fn func(context.Context) (etcdcron.Interface, error)) *Fake { +func (f *Fake) WithClient(fn func(context.Context) (api.Interface, error)) *Fake { f.clientFn = fn return f } @@ -50,6 +50,6 @@ func (f *Fake) Run(ctx context.Context) error { return f.runFn(ctx) } -func (f *Fake) Client(ctx context.Context) (etcdcron.Interface, error) { +func (f *Fake) Client(ctx context.Context) (api.Interface, error) { return f.clientFn(ctx) } diff --git a/pkg/scheduler/server/internal/serialize/serialize.go b/pkg/scheduler/server/internal/serialize/serialize.go index fd9495cd8ab..6b28e4bb93b 100644 --- a/pkg/scheduler/server/internal/serialize/serialize.go +++ b/pkg/scheduler/server/internal/serialize/serialize.go @@ -73,6 +73,26 @@ func (s *Serializer) FromRequest(ctx context.Context, req Request) (*Job, error) }, nil } +func (s *Serializer) PrefixFromList(ctx context.Context, req *schedulerv1pb.JobMetadata) (string, error) { + if err := s.authz.Metadata(ctx, req); err != nil { + return "", err + } + + switch t := req.GetTarget(); t.GetType().(type) { + case *schedulerv1pb.JobTargetMetadata_Actor: + actor := t.GetActor() + s := joinStrings("actorreminder", req.GetNamespace(), actor.GetType()) + if len(actor.GetId()) > 0 { + s = joinStrings(s, actor.GetId()) + } + return s, nil + case *schedulerv1pb.JobTargetMetadata_Job: + return joinStrings("app", req.GetNamespace(), req.GetAppId()), nil + default: + return "", fmt.Errorf("unknown job type: %v", t) + } +} + func (s *Serializer) FromWatch(stream schedulerv1pb.Scheduler_WatchJobsServer) (*schedulerv1pb.WatchJobsRequestInitial, error) { req, err := stream.Recv() if err != nil { @@ -100,10 +120,6 @@ func (j *Job) Metadata() *anypb.Any { } func buildJobName(name string, meta *schedulerv1pb.JobMetadata) (string, error) { - joinStrings := func(ss ...string) string { - return strings.Join(ss, "||") - } - switch t := meta.GetTarget(); t.GetType().(type) { case *schedulerv1pb.JobTargetMetadata_Actor: actor := t.GetActor() @@ -114,3 +130,7 @@ func buildJobName(name string, meta *schedulerv1pb.JobMetadata) (string, error) return "", fmt.Errorf("unknown job type: %v", t) } } + +func joinStrings(ss ...string) string { + return strings.Join(ss, "||") +} diff --git a/pkg/scheduler/server/server.go b/pkg/scheduler/server/server.go index 9990db14bf4..b06ed0ec337 100644 --- a/pkg/scheduler/server/server.go +++ b/pkg/scheduler/server/server.go @@ -22,13 +22,6 @@ import ( "sync" "sync/atomic" -<<<<<<< HEAD - "github.com/diagridio/go-etcd-cron/api" - "github.com/diagridio/go-etcd-cron/cron" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" -======= ->>>>>>> 4735e839d (Scheduler: Adds Kubernetes namespace controller) "google.golang.org/grpc" "github.com/dapr/dapr/pkg/healthz" @@ -73,17 +66,10 @@ type Server struct { port int sec security.Handler -<<<<<<< HEAD - authz *authz.Authz - config *embed.Config - cron api.Interface - connectionPool *internal.Pool // Connection pool for sidecars -======= serializer *serialize.Serializer cron cron.Interface connectionPool *pool.Pool // Connection pool for sidecars controller concurrency.Runner ->>>>>>> 4735e839d (Scheduler: Adds Kubernetes namespace controller) hzAPIServer healthz.Target @@ -197,73 +183,3 @@ func (s *Server) runServer(ctx context.Context) error { }, ).Run(ctx) } -<<<<<<< HEAD - -func (s *Server) runEtcdCron(ctx context.Context) error { - defer s.hzETCD.NotReady() - - log.Info("Starting etcd") - - etcd, err := embed.StartEtcd(s.config) - if err != nil { - return err - } - defer etcd.Close() - - select { - case <-etcd.Server.ReadyNotify(): - log.Info("Etcd server is ready!") - case <-ctx.Done(): - return ctx.Err() - } - - log.Info("Starting EtcdCron") - - endpoints := make([]string, 0, len(etcd.Clients)) - for _, peer := range etcd.Clients { - endpoints = append(endpoints, peer.Addr().String()) - } - - client, err := clientv3.New(clientv3.Config{ - Endpoints: endpoints, - MaxCallRecvMsgSize: math.MaxInt32, - }) - if err != nil { - return err - } - - // pass in initial cluster endpoints, but with client ports - s.cron, err = cron.New(cron.Options{ - Client: client, - Namespace: "dapr", - PartitionID: s.replicaID, - PartitionTotal: s.replicaCount, - TriggerFn: s.triggerJob, - }) - if err != nil { - return fmt.Errorf("fail to create etcd-cron: %s", err) - } - close(s.readyCh) - - s.hzETCD.Ready() - - return concurrency.NewRunnerManager( - func(ctx context.Context) error { - if err := s.cron.Run(ctx); !errors.Is(err, context.DeadlineExceeded) { - return err - } - return nil - }, - func(ctx context.Context) error { - defer log.Info("EtcdCron shutting down") - select { - case err := <-etcd.Err(): - return err - case <-ctx.Done(): - return nil - } - }, - ).Run(ctx) -} -======= ->>>>>>> 4735e839d (Scheduler: Adds Kubernetes namespace controller) diff --git a/tests/integration/framework/process/scheduler/scheduler.go b/tests/integration/framework/process/scheduler/scheduler.go index 4a2008cfc6c..132ab77e81d 100644 --- a/tests/integration/framework/process/scheduler/scheduler.go +++ b/tests/integration/framework/process/scheduler/scheduler.go @@ -361,7 +361,7 @@ func (s *Scheduler) ETCDClient(t *testing.T) *clientv3.Client { func (s *Scheduler) EtcdJobs(t *testing.T, ctx context.Context) []*mvccpb.KeyValue { t.Helper() - resp, err := s.EtcdClient(t).KV.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + resp, err := s.ETCDClient(t).KV.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) require.NoError(t, err) return resp.Kvs } From 5df95cbc194fe3620cc04df8fd4df658ec99ce70 Mon Sep 17 00:00:00 2001 From: joshvanl Date: Wed, 18 Sep 2024 15:40:33 +0100 Subject: [PATCH 05/27] Increase timeout for job be in Etcd Signed-off-by: joshvanl --- tests/integration/suite/daprd/jobs/kubernetes/namespace.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go index 09d4d168bcf..b9843420a6d 100644 --- a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go +++ b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go @@ -137,7 +137,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) require.NoError(t, err) assert.Len(c, resp.Kvs, 2) - }, time.Second*10, 10*time.Millisecond) + }, time.Second*20, 10*time.Millisecond) n.kubeapi.Informer().Delete(t, &corev1.Namespace{ TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, From 28d4927d68c46ee64b557294608b92b3317cdcce Mon Sep 17 00:00:00 2001 From: joshvanl Date: Fri, 4 Oct 2024 13:22:58 +0100 Subject: [PATCH 06/27] Linting Signed-off-by: joshvanl --- cmd/scheduler/options/options.go | 3 ++- pkg/scheduler/server/internal/controller/controller.go | 3 ++- pkg/scheduler/server/internal/controller/namespace_test.go | 1 - tests/integration/framework/process/scheduler/scheduler.go | 2 +- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/cmd/scheduler/options/options.go b/cmd/scheduler/options/options.go index 8837ffd7fa1..d175cb9f7c5 100644 --- a/cmd/scheduler/options/options.go +++ b/cmd/scheduler/options/options.go @@ -14,6 +14,7 @@ limitations under the License. package options import ( + "errors" "fmt" "strconv" "strings" @@ -139,7 +140,7 @@ func New(origArgs []string) (*Options, error) { if fs.Changed("kubeconfig") { if opts.Mode != string(modes.KubernetesMode) { - return nil, fmt.Errorf("kubeconfig flag is only valid in --mode=kubernetes") + return nil, errors.New("kubeconfig flag is only valid in --mode=kubernetes") } opts.KubeConfig = &opts.kubeconfig } diff --git a/pkg/scheduler/server/internal/controller/controller.go b/pkg/scheduler/server/internal/controller/controller.go index 85c9d8352d7..5b67d9d5449 100644 --- a/pkg/scheduler/server/internal/controller/controller.go +++ b/pkg/scheduler/server/internal/controller/controller.go @@ -15,6 +15,7 @@ package controller import ( "context" + "errors" "fmt" "os" @@ -98,7 +99,7 @@ func New(opts Options) (concurrency.Runner, error) { return fmt.Errorf("unable to get informer: %w", err) } if !mgr.GetCache().WaitForCacheSync(ctx) { - return fmt.Errorf("unable to sync cache") + return errors.New("unable to sync cache") } hzTarget.Ready() log.Info("Controller ready") diff --git a/pkg/scheduler/server/internal/controller/namespace_test.go b/pkg/scheduler/server/internal/controller/namespace_test.go index 51d5a899a30..76ee2919229 100644 --- a/pkg/scheduler/server/internal/controller/namespace_test.go +++ b/pkg/scheduler/server/internal/controller/namespace_test.go @@ -63,7 +63,6 @@ func Test_Reconcile(t *testing.T) { req := ctrl.Request{NamespacedName: types.NamespacedName{Name: "test-ns"}} for name, test := range tests { - test := test t.Run(name, func(t *testing.T) { t.Parallel() diff --git a/tests/integration/framework/process/scheduler/scheduler.go b/tests/integration/framework/process/scheduler/scheduler.go index 132ab77e81d..2d0475d4bc7 100644 --- a/tests/integration/framework/process/scheduler/scheduler.go +++ b/tests/integration/framework/process/scheduler/scheduler.go @@ -347,7 +347,7 @@ func (s *Scheduler) ETCDClient(t *testing.T) *clientv3.Client { t.Helper() client, err := clientv3.New(clientv3.Config{ - Endpoints: []string{fmt.Sprintf("127.0.0.1:%s", s.EtcdClientPort())}, + Endpoints: []string{"127.0.0.1:" + s.EtcdClientPort()}, DialTimeout: 40 * time.Second, }) require.NoError(t, err) From ff0e4231e8b2dd8bc00b7f3992c4e410b14b79fc Mon Sep 17 00:00:00 2001 From: joshvanl Date: Tue, 8 Oct 2024 16:19:11 +0100 Subject: [PATCH 07/27] Adds informer waits for scheduler namespace informer Signed-off-by: joshvanl --- .../process/kubernetes/informer/informer.go | 60 ++++++++++++++++++- .../suite/daprd/jobs/kubernetes/namespace.go | 2 +- .../suite/scheduler/kubernetes/namespace.go | 27 ++++----- 3 files changed, 71 insertions(+), 18 deletions(-) diff --git a/tests/integration/framework/process/kubernetes/informer/informer.go b/tests/integration/framework/process/kubernetes/informer/informer.go index b6fb5c12807..e4a0ac70e8c 100644 --- a/tests/integration/framework/process/kubernetes/informer/informer.go +++ b/tests/integration/framework/process/kubernetes/informer/informer.go @@ -14,11 +14,15 @@ limitations under the License. package informer import ( + "bytes" + "context" "encoding/json" + "math/rand/v2" "net/http" "strings" "sync" "testing" + "time" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -41,11 +45,14 @@ import ( type Informer struct { lock sync.Mutex active map[string][][]byte + + informed map[uint64]chan *metav1.WatchEvent } func New() *Informer { return &Informer{ - active: make(map[string][][]byte), + active: make(map[string][][]byte), + informed: make(map[uint64]chan *metav1.WatchEvent), } } @@ -89,8 +96,18 @@ func (i *Informer) Handler(t *testing.T, wrapped http.Handler) http.HandlerFunc w.WriteHeader(http.StatusOK) if len(i.active[gvk.String()]) > 0 { + var event metav1.WatchEvent + assert.NoError(t, json.Unmarshal(i.active[gvk.String()][0], &event)) w.Write(i.active[gvk.String()][0]) i.active[gvk.String()] = i.active[gvk.String()][1:] + + for _, ch := range i.informed { + select { + case ch <- &event: + case <-time.After(3 * time.Second): + t.Errorf("failed to send informed event to subscriber") + } + } } w.(http.Flusher).Flush() } @@ -111,6 +128,47 @@ func (i *Informer) Delete(t *testing.T, obj runtime.Object) { i.inform(t, obj, string(watch.Deleted)) } +func (i *Informer) DeleteWait(t *testing.T, ctx context.Context, obj runtime.Object) { + t.Helper() + + i.lock.Lock() + //nolint:gosec + ui := rand.Uint64() + ch := make(chan *metav1.WatchEvent) + i.informed[ui] = ch + i.lock.Unlock() + + defer func() { + i.lock.Lock() + close(ch) + delete(i.informed, ui) + i.lock.Unlock() + }() + + i.Delete(t, obj) + + exp, err := json.Marshal(obj) + require.NoError(t, err) + + for { + select { + case <-ctx.Done(): + assert.Fail(t, "failed to wait for delete event to occur") + return + case e := <-ch: + if e.Type != string(watch.Deleted) { + continue + } + + if !bytes.Equal(exp, e.Object.Raw) { + continue + } + + return + } + } +} + func (i *Informer) inform(t *testing.T, obj runtime.Object, event string) { t.Helper() i.lock.Lock() diff --git a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go index b9843420a6d..37400838ee4 100644 --- a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go +++ b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go @@ -139,7 +139,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { assert.Len(c, resp.Kvs, 2) }, time.Second*20, 10*time.Millisecond) - n.kubeapi.Informer().Delete(t, &corev1.Namespace{ + n.kubeapi.Informer().DeleteWait(t, ctx, &corev1.Namespace{ TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, ObjectMeta: metav1.ObjectMeta{Name: "default"}, }) diff --git a/tests/integration/suite/scheduler/kubernetes/namespace.go b/tests/integration/suite/scheduler/kubernetes/namespace.go index 2cdc9e985c3..ac1814cb4f0 100644 --- a/tests/integration/suite/scheduler/kubernetes/namespace.go +++ b/tests/integration/suite/scheduler/kubernetes/namespace.go @@ -27,7 +27,6 @@ import ( schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" "github.com/dapr/dapr/tests/integration/framework" "github.com/dapr/dapr/tests/integration/framework/process/kubernetes" - "github.com/dapr/dapr/tests/integration/framework/process/kubernetes/store" "github.com/dapr/dapr/tests/integration/framework/process/scheduler" "github.com/dapr/dapr/tests/integration/framework/process/sentry" "github.com/dapr/dapr/tests/integration/suite" @@ -41,33 +40,29 @@ func init() { type namespace struct { sentry *sentry.Sentry scheduler *scheduler.Scheduler - store *store.Store + kubeapi *kubernetes.Kubernetes } func (n *namespace) Setup(t *testing.T) []framework.Option { n.sentry = sentry.New(t) - n.store = store.New(metav1.GroupVersionKind{ - Version: "v1", - Kind: "Namespace", - }) - n.store.Add(&corev1.Namespace{ - TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, - ObjectMeta: metav1.ObjectMeta{Name: "default"}, - }) - - kubeapi := kubernetes.New(t, - kubernetes.WithClusterNamespaceListFromStore(t, n.store), + n.kubeapi = kubernetes.New(t, + kubernetes.WithClusterNamespaceList(t, &corev1.NamespaceList{ + Items: []corev1.Namespace{{ + TypeMeta: metav1.TypeMeta{Kind: "Namespace", APIVersion: "v1"}, + ObjectMeta: metav1.ObjectMeta{Name: "default"}, + }}, + }), ) n.scheduler = scheduler.New(t, scheduler.WithSentry(n.sentry), - scheduler.WithKubeconfig(kubeapi.KubeconfigPath(t)), + scheduler.WithKubeconfig(n.kubeapi.KubeconfigPath(t)), scheduler.WithMode("kubernetes"), ) return []framework.Option{ - framework.WithProcesses(n.sentry, kubeapi, n.scheduler), + framework.WithProcesses(n.sentry, n.kubeapi, n.scheduler), } } @@ -114,7 +109,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { assert.Len(c, resp.Kvs, 2) }, time.Second*10, 10*time.Millisecond) - n.store.Delete(&corev1.Namespace{ + n.kubeapi.Informer().DeleteWait(t, ctx, &corev1.Namespace{ TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: "Namespace"}, ObjectMeta: metav1.ObjectMeta{Name: "default"}, }) From 743f4f4af86483cd0d54c17895ee9b737a7c881f Mon Sep 17 00:00:00 2001 From: joshvanl Date: Tue, 8 Oct 2024 16:53:11 +0100 Subject: [PATCH 08/27] Use filepath join for Etcd get path to account for windows Signed-off-by: joshvanl --- tests/integration/suite/daprd/jobs/kubernetes/namespace.go | 5 +++-- tests/integration/suite/scheduler/kubernetes/namespace.go | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go index 37400838ee4..637c4a67bd7 100644 --- a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go +++ b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go @@ -16,6 +16,7 @@ package kubernetes import ( "context" "net/http" + "path/filepath" "testing" "time" @@ -134,7 +135,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { etcdClient := n.scheduler.ETCDClient(t).KV assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, filepath.Join("dapr", "jobs"), clientv3.WithPrefix()) require.NoError(t, err) assert.Len(c, resp.Kvs, 2) }, time.Second*20, 10*time.Millisecond) @@ -145,7 +146,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { }) assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, filepath.Join("dapr", "jobs"), clientv3.WithPrefix()) require.NoError(t, err) assert.Empty(c, resp.Kvs) }, time.Second*10, 10*time.Millisecond) diff --git a/tests/integration/suite/scheduler/kubernetes/namespace.go b/tests/integration/suite/scheduler/kubernetes/namespace.go index ac1814cb4f0..9b973568985 100644 --- a/tests/integration/suite/scheduler/kubernetes/namespace.go +++ b/tests/integration/suite/scheduler/kubernetes/namespace.go @@ -15,6 +15,7 @@ package kubernetes import ( "context" + "path/filepath" "testing" "time" @@ -104,7 +105,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { etcdClient := n.scheduler.ETCDClient(t).KV assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, filepath.Join("dapr", "jobs"), clientv3.WithPrefix()) require.NoError(t, err) assert.Len(c, resp.Kvs, 2) }, time.Second*10, 10*time.Millisecond) @@ -115,7 +116,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { }) assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err := etcdClient.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, filepath.Join("dapr", "jobs"), clientv3.WithPrefix()) require.NoError(t, err) assert.Empty(c, resp.Kvs) }, time.Second*10, 10*time.Millisecond) From 2f3d97c4f91f2f615f42b3e45736650b7622fd7d Mon Sep 17 00:00:00 2001 From: joshvanl Date: Thu, 10 Oct 2024 23:44:23 +0100 Subject: [PATCH 09/27] Review comments Signed-off-by: joshvanl --- cmd/scheduler/options/options.go | 3 + .../server/internal/serialize/serialize.go | 11 +- .../internal/serialize/serialize_test.go | 102 ++++++++++++++++++ .../process/kubernetes/informer/informer.go | 2 + .../process/kubernetes/store/store.go | 1 + 5 files changed, 117 insertions(+), 2 deletions(-) create mode 100644 pkg/scheduler/server/internal/serialize/serialize_test.go diff --git a/cmd/scheduler/options/options.go b/cmd/scheduler/options/options.go index d175cb9f7c5..e8d6c09de43 100644 --- a/cmd/scheduler/options/options.go +++ b/cmd/scheduler/options/options.go @@ -87,6 +87,9 @@ func New(origArgs []string) (*Options, error) { fs.StringVar(&opts.SentryAddress, "sentry-address", fmt.Sprintf("dapr-sentry.%s.svc:443", security.CurrentNamespace()), "Address of the Sentry service") fs.StringVar(&opts.Mode, "mode", string(modes.StandaloneMode), "Runtime mode for Dapr Scheduler") fs.StringVar(&opts.kubeconfig, "kubeconfig", "", "Kubernetes mode only. Absolute path to the kubeconfig file.") + if err := fs.MarkHidden("kubeconfig"); err != nil { + panic(err) + } fs.StringVar(&opts.ID, "id", "dapr-scheduler-server-0", "Scheduler server ID") fs.Uint32Var(&opts.ReplicaCount, "replica-count", 1, "The total number of scheduler replicas in the cluster") diff --git a/pkg/scheduler/server/internal/serialize/serialize.go b/pkg/scheduler/server/internal/serialize/serialize.go index 6b28e4bb93b..842361f2f76 100644 --- a/pkg/scheduler/server/internal/serialize/serialize.go +++ b/pkg/scheduler/server/internal/serialize/serialize.go @@ -62,7 +62,7 @@ func (s *Serializer) FromRequest(ctx context.Context, req Request) (*Job, error) return nil, err } - name, err := buildJobName(req.GetName(), req.GetMetadata()) + name, err := buildJobName(req) if err != nil { return nil, err } @@ -119,7 +119,14 @@ func (j *Job) Metadata() *anypb.Any { return j.meta } -func buildJobName(name string, meta *schedulerv1pb.JobMetadata) (string, error) { +func buildJobName(req Request) (string, error) { + joinStrings := func(ss ...string) string { + return strings.Join(ss, "||") + } + + name := req.GetName() + meta := req.GetMetadata() + switch t := meta.GetTarget(); t.GetType().(type) { case *schedulerv1pb.JobTargetMetadata_Actor: actor := t.GetActor() diff --git a/pkg/scheduler/server/internal/serialize/serialize_test.go b/pkg/scheduler/server/internal/serialize/serialize_test.go new file mode 100644 index 00000000000..a5867ba7fff --- /dev/null +++ b/pkg/scheduler/server/internal/serialize/serialize_test.go @@ -0,0 +1,102 @@ +/* +Copyright 2024 The Dapr 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 serialize + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" +) + +func Test_buildJobName(t *testing.T) { + t.Parallel() + + tests := map[string]struct { + req Request + expName string + expErr bool + }{ + "nil meta should return error": { + req: &schedulerv1pb.ScheduleJobRequest{ + Name: "test", + Metadata: nil, + }, + expName: "", + expErr: true, + }, + "meta with nil target should return error": { + req: &schedulerv1pb.ScheduleJobRequest{ + Name: "test", + Metadata: new(schedulerv1pb.JobMetadata), + }, + expName: "", + expErr: true, + }, + "job meta should return concatenated name": { + req: &schedulerv1pb.ScheduleJobRequest{ + Name: "test", + Metadata: &schedulerv1pb.JobMetadata{ + Namespace: "myns", AppId: "myapp", + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Job{ + Job: new(schedulerv1pb.TargetJob), + }, + }}, + }, + expName: "app||myns||myapp||test", + expErr: false, + }, + "nil job meta should return concatenated name": { + req: &schedulerv1pb.ScheduleJobRequest{ + Name: "test", + Metadata: &schedulerv1pb.JobMetadata{ + Namespace: "myns", AppId: "myapp", + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Job{ + Job: nil, + }, + }}, + }, + expName: "app||myns||myapp||test", + expErr: false, + }, + "actor meta should return concatenated name": { + req: &schedulerv1pb.ScheduleJobRequest{ + Name: "test", + Metadata: &schedulerv1pb.JobMetadata{ + Namespace: "myns", AppId: "myapp", + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Actor{ + Actor: &schedulerv1pb.TargetActorReminder{ + Type: "myactortype", Id: "myactorid", + }, + }, + }}, + }, + expName: "actorreminder||myns||myactortype||myactorid||test", + expErr: false, + }, + } + + for name, test := range tests { + t.Run(name, func(t *testing.T) { + t.Parallel() + got, err := buildJobName(test.req) + assert.Equal(t, err != nil, test.expErr) + assert.Equal(t, test.expName, got) + }) + } +} diff --git a/tests/integration/framework/process/kubernetes/informer/informer.go b/tests/integration/framework/process/kubernetes/informer/informer.go index e4a0ac70e8c..5cb455196c9 100644 --- a/tests/integration/framework/process/kubernetes/informer/informer.go +++ b/tests/integration/framework/process/kubernetes/informer/informer.go @@ -81,6 +81,8 @@ func (i *Informer) Handler(t *testing.T, wrapped http.Handler) http.HandlerFunc split = split[2:] } if split[0] == "namespaces" { + // namespace resources are special cased in the Kubernetes CRUD resource + // URL, so we need to handle them differently. if len(split) > 1 { split = split[2:] gvk.Kind = split[0] diff --git a/tests/integration/framework/process/kubernetes/store/store.go b/tests/integration/framework/process/kubernetes/store/store.go index 9d914187002..88a861ca152 100644 --- a/tests/integration/framework/process/kubernetes/store/store.go +++ b/tests/integration/framework/process/kubernetes/store/store.go @@ -50,6 +50,7 @@ func (s *Store) Delete(objs ...client.Object) { defer s.lock.Unlock() if s.objs == nil { s.objs = make(map[string]client.Object) + return } for _, obj := range objs { delete(s.objs, obj.GetNamespace()+"/"+obj.GetName()) From 0182c0f874b4eb4e61b7c1b776c09b67a8fbbaeb Mon Sep 17 00:00:00 2001 From: joshvanl Date: Wed, 16 Oct 2024 16:42:47 +0100 Subject: [PATCH 10/27] Fix scheduler trigger metrics Signed-off-by: joshvanl --- .../server/internal/controller/namespace_test.go | 4 ++-- pkg/scheduler/server/internal/cron/cron.go | 6 ++++++ pkg/scheduler/server/internal/serialize/serialize.go | 4 ---- .../server/internal/serialize/serialize_test.go | 11 +++++++---- 4 files changed, 15 insertions(+), 10 deletions(-) diff --git a/pkg/scheduler/server/internal/controller/namespace_test.go b/pkg/scheduler/server/internal/controller/namespace_test.go index 76ee2919229..19d9be7dc22 100644 --- a/pkg/scheduler/server/internal/controller/namespace_test.go +++ b/pkg/scheduler/server/internal/controller/namespace_test.go @@ -17,7 +17,7 @@ import ( "context" "testing" - "github.com/diagridio/go-etcd-cron/cron" + "github.com/diagridio/go-etcd-cron/api" etcdcronfake "github.com/diagridio/go-etcd-cron/tests/fake" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -67,7 +67,7 @@ func Test_Reconcile(t *testing.T) { t.Parallel() var prefixCalled []string - cron := cronfake.New().WithClient(func(context.Context) (cron.Interface, error) { + cron := cronfake.New().WithClient(func(context.Context) (api.Interface, error) { etcdcron := etcdcronfake.New().WithDeletePrefixes(func(_ context.Context, prefixes ...string) error { prefixCalled = append(prefixCalled, prefixes...) return test.deletePError diff --git a/pkg/scheduler/server/internal/cron/cron.go b/pkg/scheduler/server/internal/cron/cron.go index 71869813f66..36c9d1d3e96 100644 --- a/pkg/scheduler/server/internal/cron/cron.go +++ b/pkg/scheduler/server/internal/cron/cron.go @@ -27,6 +27,7 @@ import ( "github.com/dapr/dapr/pkg/healthz" schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" + "github.com/dapr/dapr/pkg/scheduler/monitoring" "github.com/dapr/dapr/pkg/scheduler/server/internal/pool" "github.com/dapr/kit/concurrency" "github.com/dapr/kit/logger" @@ -170,6 +171,7 @@ func (c *cron) triggerJob(ctx context.Context, req *api.TriggerRequest) bool { return true } + now := time.Now() if err := c.connectionPool.Send(ctx, &pool.JobEvent{ Name: req.GetName()[idx+2:], Data: req.GetPayload(), @@ -180,5 +182,9 @@ func (c *cron) triggerJob(ctx context.Context, req *api.TriggerRequest) bool { log.Errorf("Error sending job to connection stream: %s", err) } + monitoring.RecordTriggerDuration(now) + + monitoring.RecordJobsTriggeredCount(&meta) + return true } diff --git a/pkg/scheduler/server/internal/serialize/serialize.go b/pkg/scheduler/server/internal/serialize/serialize.go index 842361f2f76..d48678f9a4e 100644 --- a/pkg/scheduler/server/internal/serialize/serialize.go +++ b/pkg/scheduler/server/internal/serialize/serialize.go @@ -120,10 +120,6 @@ func (j *Job) Metadata() *anypb.Any { } func buildJobName(req Request) (string, error) { - joinStrings := func(ss ...string) string { - return strings.Join(ss, "||") - } - name := req.GetName() meta := req.GetMetadata() diff --git a/pkg/scheduler/server/internal/serialize/serialize_test.go b/pkg/scheduler/server/internal/serialize/serialize_test.go index a5867ba7fff..7aa61293f3c 100644 --- a/pkg/scheduler/server/internal/serialize/serialize_test.go +++ b/pkg/scheduler/server/internal/serialize/serialize_test.go @@ -54,7 +54,8 @@ func Test_buildJobName(t *testing.T) { Type: &schedulerv1pb.JobTargetMetadata_Job{ Job: new(schedulerv1pb.TargetJob), }, - }}, + }, + }, }, expName: "app||myns||myapp||test", expErr: false, @@ -68,7 +69,8 @@ func Test_buildJobName(t *testing.T) { Type: &schedulerv1pb.JobTargetMetadata_Job{ Job: nil, }, - }}, + }, + }, }, expName: "app||myns||myapp||test", expErr: false, @@ -84,7 +86,8 @@ func Test_buildJobName(t *testing.T) { Type: "myactortype", Id: "myactorid", }, }, - }}, + }, + }, }, expName: "actorreminder||myns||myactortype||myactorid||test", expErr: false, @@ -95,7 +98,7 @@ func Test_buildJobName(t *testing.T) { t.Run(name, func(t *testing.T) { t.Parallel() got, err := buildJobName(test.req) - assert.Equal(t, err != nil, test.expErr) + assert.Equal(t, test.expErr, err != nil) assert.Equal(t, test.expName, got) }) } From 672d49f0d596feb226c5c6212e4f008c74b762b0 Mon Sep 17 00:00:00 2001 From: joshvanl Date: Wed, 16 Oct 2024 17:21:33 +0100 Subject: [PATCH 11/27] go mod tidy Signed-off-by: joshvanl --- go.mod | 1 - go.sum | 2 -- 2 files changed, 3 deletions(-) diff --git a/go.mod b/go.mod index 35274102a3d..61ff61a94a6 100644 --- a/go.mod +++ b/go.mod @@ -68,7 +68,6 @@ require ( golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 golang.org/x/net v0.29.0 golang.org/x/sync v0.8.0 - google.golang.org/appengine v1.6.8 google.golang.org/genproto/googleapis/api v0.0.0-20240924160255-9d4c2d233b61 google.golang.org/genproto/googleapis/rpc v0.0.0-20240924160255-9d4c2d233b61 google.golang.org/grpc v1.67.0 diff --git a/go.sum b/go.sum index 6d0c5c7f2b5..c7533ce959e 100644 --- a/go.sum +++ b/go.sum @@ -2272,8 +2272,6 @@ google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= -google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20170818010345-ee236bd376b0/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= From db3d10e32a37d8de97fdd9006aae461a778d34e8 Mon Sep 17 00:00:00 2001 From: joshvanl Date: Wed, 16 Oct 2024 17:56:40 +0100 Subject: [PATCH 12/27] Construct Etcd key paths manually, instead of legacy filepath.Join Signed-off-by: joshvanl --- tests/integration/suite/daprd/jobs/kubernetes/namespace.go | 5 ++--- tests/integration/suite/scheduler/kubernetes/namespace.go | 5 ++--- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go index 637c4a67bd7..065f5f71a8a 100644 --- a/tests/integration/suite/daprd/jobs/kubernetes/namespace.go +++ b/tests/integration/suite/daprd/jobs/kubernetes/namespace.go @@ -16,7 +16,6 @@ package kubernetes import ( "context" "net/http" - "path/filepath" "testing" "time" @@ -135,7 +134,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { etcdClient := n.scheduler.ETCDClient(t).KV assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err := etcdClient.Get(ctx, filepath.Join("dapr", "jobs"), clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, "dapr/jobs/", clientv3.WithPrefix()) require.NoError(t, err) assert.Len(c, resp.Kvs, 2) }, time.Second*20, 10*time.Millisecond) @@ -146,7 +145,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { }) assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err := etcdClient.Get(ctx, filepath.Join("dapr", "jobs"), clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, "dapr/jobs/", clientv3.WithPrefix()) require.NoError(t, err) assert.Empty(c, resp.Kvs) }, time.Second*10, 10*time.Millisecond) diff --git a/tests/integration/suite/scheduler/kubernetes/namespace.go b/tests/integration/suite/scheduler/kubernetes/namespace.go index 9b973568985..3462edb6e02 100644 --- a/tests/integration/suite/scheduler/kubernetes/namespace.go +++ b/tests/integration/suite/scheduler/kubernetes/namespace.go @@ -15,7 +15,6 @@ package kubernetes import ( "context" - "path/filepath" "testing" "time" @@ -105,7 +104,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { etcdClient := n.scheduler.ETCDClient(t).KV assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err := etcdClient.Get(ctx, filepath.Join("dapr", "jobs"), clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, "dapr/jobs/", clientv3.WithPrefix()) require.NoError(t, err) assert.Len(c, resp.Kvs, 2) }, time.Second*10, 10*time.Millisecond) @@ -116,7 +115,7 @@ func (n *namespace) Run(t *testing.T, ctx context.Context) { }) assert.EventuallyWithT(t, func(c *assert.CollectT) { - resp, err := etcdClient.Get(ctx, filepath.Join("dapr", "jobs"), clientv3.WithPrefix()) + resp, err := etcdClient.Get(ctx, "dapr/jobs/", clientv3.WithPrefix()) require.NoError(t, err) assert.Empty(c, resp.Kvs) }, time.Second*10, 10*time.Millisecond) From 027155e0fd30e86d1d65f56fb16b75ded72b9bf7 Mon Sep 17 00:00:00 2001 From: Elena Kolevska Date: Thu, 17 Oct 2024 15:41:43 +0100 Subject: [PATCH 13/27] Fixes flaky `scheduler/idtype` and `TestMonitorLeadership` tests (#7973) * Fixes using testing.T instead of assert.Collect Signed-off-by: Elena Kolevska * Accounts for internal raft leadership transfer timeout in test Signed-off-by: Elena Kolevska --------- Signed-off-by: Elena Kolevska Co-authored-by: Yaron Schneider --- pkg/placement/leadership_test.go | 7 ++++++- .../suite/actors/reminders/scheduler/idtypes.go | 2 +- .../integration/suite/actors/reminders/scheduler/remote.go | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/pkg/placement/leadership_test.go b/pkg/placement/leadership_test.go index 875327a5fbb..d6934153fb6 100644 --- a/pkg/placement/leadership_test.go +++ b/pkg/placement/leadership_test.go @@ -97,7 +97,12 @@ func TestMonitorLeadership(t *testing.T) { }, time.Second*15, 10*time.Millisecond, "raft server two was not set as leader in time") // Transfer leadership back to the original leader - underlyingRaftServers[secondServerID].LeadershipTransferToServer(hashicorpRaft.ServerID(raftServers[leaderIdx].GetID()), hashicorpRaft.ServerAddress(raftServers[leaderIdx].GetRaftBind())) + var future hashicorpRaft.Future + require.Eventually(t, func() bool { + future = underlyingRaftServers[secondServerID].LeadershipTransferToServer(hashicorpRaft.ServerID(raftServers[leaderIdx].GetID()), hashicorpRaft.ServerAddress(raftServers[leaderIdx].GetRaftBind())) + return future.Error() == nil + }, time.Second*15, 500*time.Millisecond, "raft leadership transfer timeout") + require.Eventually(t, func() bool { return raftServers[leaderIdx].IsLeader() && placementServers[leaderIdx].hasLeadership.Load() }, time.Second*15, 10*time.Millisecond, "server was not properly re-elected in time") diff --git a/tests/integration/suite/actors/reminders/scheduler/idtypes.go b/tests/integration/suite/actors/reminders/scheduler/idtypes.go index 9b523cf6c22..e44c198c206 100644 --- a/tests/integration/suite/actors/reminders/scheduler/idtypes.go +++ b/tests/integration/suite/actors/reminders/scheduler/idtypes.go @@ -167,7 +167,7 @@ func (i *idtype) Run(t *testing.T, ctx context.Context) { require.EventuallyWithT(t, func(c *assert.CollectT) { invoke := fmt.Sprintf("%s/%s/%s/method/foo", daprdURL, i.actorDaprds[x].actorTypes[y].typename, i.actorDaprds[x].actorTypes[y].ids[z]) req, err := http.NewRequestWithContext(ctx, http.MethodPost, invoke, nil) - require.NoError(t, err) + require.NoError(c, err) resp, err := client.Do(req) if assert.NoError(c, err) { assert.NoError(c, resp.Body.Close()) diff --git a/tests/integration/suite/actors/reminders/scheduler/remote.go b/tests/integration/suite/actors/reminders/scheduler/remote.go index 1f73dccfdd8..44ee503f4b1 100644 --- a/tests/integration/suite/actors/reminders/scheduler/remote.go +++ b/tests/integration/suite/actors/reminders/scheduler/remote.go @@ -155,7 +155,7 @@ func (r *remote) Run(t *testing.T, ctx context.Context) { }, time.Second*5, time.Millisecond*10) require.EventuallyWithT(t, func(c *assert.CollectT) { - assert.ElementsMatch(t, r.actorIDs, r.methodcalled.Load().([]string)) + assert.ElementsMatch(c, r.actorIDs, r.methodcalled.Load().([]string)) }, time.Second*10, time.Millisecond*10) assert.GreaterOrEqual(t, r.daprd1called.Load(), uint64(0)) From 2cbded5c79f113171bb107d408e1ca6f1a80bf0e Mon Sep 17 00:00:00 2001 From: Elena Kolevska Date: Fri, 18 Oct 2024 22:56:49 +0100 Subject: [PATCH 14/27] Fix flakey raft ha test (#8197) * Fixes nil pointer Signed-off-by: Elena Kolevska linter Signed-off-by: Elena Kolevska handles timeout Signed-off-by: Elena Kolevska * Removes time sleep Signed-off-by: Elena Kolevska * removes dummy line that caused flakiness Signed-off-by: Elena Kolevska * removes time sleep Signed-off-by: Elena Kolevska * timeout tweak Signed-off-by: Elena Kolevska * Extra check for good measure Signed-off-by: Elena Kolevska --------- Signed-off-by: Elena Kolevska --- pkg/placement/raft/ha_test.go | 71 +++++++++++------------------------ 1 file changed, 22 insertions(+), 49 deletions(-) diff --git a/pkg/placement/raft/ha_test.go b/pkg/placement/raft/ha_test.go index fa19cb0de8a..b0539e968c5 100644 --- a/pkg/placement/raft/ha_test.go +++ b/pkg/placement/raft/ha_test.go @@ -79,13 +79,6 @@ func TestRaftHA(t *testing.T) { // Run tests t.Run("elects leader with 3 nodes", func(t *testing.T) { - // It is painful that we have to include a `time.Sleep` here, but due to - // the non-deterministic behaviour of the raft library we are using we will - // later fail on slower test runner machines. A clock timer wait means we - // have a _better_ chance of being in the right spot in the state machine - // and the network has died down. Ideally we should move to a different - // raft library that is more deterministic and reliable for our use case. - time.Sleep(time.Second * 3) require.NotEqual(t, -1, findLeader(t, raftServers)) }) @@ -110,20 +103,14 @@ func TestRaftHA(t *testing.T) { follower = (oldLeader + 1) % 3 retrieveValidState(t, raftServers[follower], testMembers[0]) - t.Run("new leader after leader fails", func(t *testing.T) { + t.Run("new leader is elected after leader fails", func(t *testing.T) { + // Stop the current leader raftServerCancel[oldLeader]() raftServers[oldLeader] = nil - // It is painful that we have to include a `time.Sleep` here, but due to - // the non-deterministic behaviour of the raft library we are using we will - // later fail on slower test runner machines. A clock timer wait means we - // have a _better_ chance of being in the right spot in the state machine - // and the network has died down. Ideally we should move to a different - // raft library that is more deterministic and reliable for our use case. - time.Sleep(time.Second * 3) - require.Eventually(t, func() bool { - return oldLeader != findLeader(t, raftServers) + newLeader := findLeader(t, raftServers) + return oldLeader != newLeader && newLeader != -1 }, time.Second*10, time.Millisecond*100) }) }) @@ -143,13 +130,6 @@ func TestRaftHA(t *testing.T) { }) t.Run("leave only leader node running", func(t *testing.T) { - // It is painful that we have to include a `time.Sleep` here, but due to - // the non-deterministic behaviour of the raft library we are using we will - // fail in a few lines on slower test runner machines. A clock timer wait - // means we have a _better_ chance of being in the right spot in the state - // machine. Ideally we should move to a different raft library that is more - // deterministic and reliable. - time.Sleep(time.Second * 3) leader := findLeader(t, raftServers) for i := range raftServers { if i != leader { @@ -177,22 +157,16 @@ func TestRaftHA(t *testing.T) { }, time.Second*5, time.Millisecond*100, "leader did not step down") }) - // It is painful that we have to include a `time.Sleep` here, but due to - // the non-deterministic behaviour of the raft library we are using we will - // fail in a few lines on slower test runner machines. A clock timer wait - // means we have a _better_ chance of being in the right spot in the state - // machine. Ideally we should move to a different raft library that is more - // deterministic and reliable. - time.Sleep(time.Second * 6) - t.Run("leader elected when second node comes up", func(t *testing.T) { - for oldSvr := range 3 { - if raftServers[oldSvr] == nil { + oldSvr := -1 + for i := range 3 { + if raftServers[i] == nil { + oldSvr = i break } } + require.NotEqual(t, -1, oldSvr, "no server to replace") - oldSvr := 2 raftServers[oldSvr], ready[oldSvr], raftServerCancel[oldSvr] = createRaftServer(t, oldSvr, peers) select { case <-ready[oldSvr]: @@ -249,14 +223,6 @@ func TestRaftHA(t *testing.T) { } } - // It is painful that we have to include a `time.Sleep` here, but due to - // the non-deterministic behaviour of the raft library we are using we will - // later fail on slower test runner machines. A clock timer wait means we - // have a _better_ chance of being in the right spot in the state machine - // and the network has died down. Ideally we should move to a different - // raft library that is more deterministic and reliable for our use case. - time.Sleep(time.Second * 3) - // Restart all nodes for i := range 3 { raftServers[i], ready[i], raftServerCancel[i] = createRaftServer(t, i, peers) @@ -322,19 +288,26 @@ func createRaftServer(t *testing.T, nodeID int, peers []PeerInfo) (*Server, <-ch go func() { defer close(ready) for { - timeoutCtx, timeoutCancel := context.WithTimeout(ctx, time.Second*5) - defer timeoutCancel() - r, err := srv.Raft(timeoutCtx) - assert.NoError(t, err) - if r.State() == hcraft.Follower || r.State() == hcraft.Leader { + select { + case <-ctx.Done(): return + default: + timeoutCtx, timeoutCancel := context.WithTimeout(ctx, time.Second*5) + r, err := srv.Raft(timeoutCtx) + if err == nil && (r.State() == hcraft.Follower || r.State() == hcraft.Leader) { + timeoutCancel() + return + } + timeoutCancel() } } }() + // Advance the clock to trigger elections more quickly go func() { for { select { + case <-ctx.Done(): case <-ready: case <-time.After(time.Millisecond): clock.Step(time.Second * 2) @@ -375,7 +348,7 @@ func findLeader(t *testing.T, raftServers []*Server) int { } return true - }, time.Second*30, time.Second, "no leader elected") + }, time.Second*30, 500*time.Millisecond, "no leader elected") return n } From 990aa6eb1483258f0b9f9e0612ecb82204665e53 Mon Sep 17 00:00:00 2001 From: Ryan Kelly Date: Fri, 18 Oct 2024 17:42:49 -0500 Subject: [PATCH 15/27] overwrite dapr-api-token when sending local instead of appending (#8173) Signed-off-by: Ryan Kelly Co-authored-by: Yaron Schneider Co-authored-by: Mike Nguyen Co-authored-by: Artur Souza --- pkg/messaging/grpc_proxy.go | 7 +++++-- pkg/messaging/grpc_proxy_test.go | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/pkg/messaging/grpc_proxy.go b/pkg/messaging/grpc_proxy.go index 853c0de39d6..2372a1e3596 100644 --- a/pkg/messaging/grpc_proxy.go +++ b/pkg/messaging/grpc_proxy.go @@ -103,7 +103,6 @@ func (p *proxy) intercept(ctx context.Context, fullName string) (context.Context return ctx, nil, nil, nopTeardown, fmt.Errorf("failed to proxy request: required metadata %s not found", diagnostics.GRPCProxyAppIDKey) } - outCtx := metadata.NewOutgoingContext(ctx, md.Copy()) appID := v[0] if p.remoteAppFn == nil { @@ -132,12 +131,16 @@ func (p *proxy) intercept(ctx context.Context, fullName string) (context.Context appMetadataToken := security.GetAppToken() if appMetadataToken != "" { - outCtx = metadata.AppendToOutgoingContext(outCtx, securityConsts.APITokenHeader, appMetadataToken) + md.Set(securityConsts.APITokenHeader, appMetadataToken) } + outCtx := metadata.NewOutgoingContext(ctx, md.Copy()) + return outCtx, appClient.(*grpc.ClientConn), nil, nopTeardown, nil } + outCtx := metadata.NewOutgoingContext(ctx, md.Copy()) + // proxy to a remote daprd conn, teardown, cErr := p.connectionFactory(outCtx, target.address, target.id, target.namespace, grpc.WithDefaultCallOptions(grpc.CallContentSubtype((&codec.Proxy{}).Name())), diff --git a/pkg/messaging/grpc_proxy_test.go b/pkg/messaging/grpc_proxy_test.go index 1093cdad46c..483d312c9ac 100644 --- a/pkg/messaging/grpc_proxy_test.go +++ b/pkg/messaging/grpc_proxy_test.go @@ -190,7 +190,7 @@ func TestIntercept(t *testing.T) { t.Setenv(securityConsts.AppAPITokenEnvVar, "token1") - ctx := metadata.NewIncomingContext(context.TODO(), metadata.MD{diagnostics.GRPCProxyAppIDKey: []string{"a"}}) + ctx := metadata.NewIncomingContext(context.TODO(), metadata.MD{diagnostics.GRPCProxyAppIDKey: []string{"a"}, securityConsts.APITokenHeader: []string{"oldtoken"}}) proxy := p.(*proxy) ctx, conn, _, teardown, err := proxy.intercept(ctx, "/test") defer teardown(true) From 686e0617f93389aa8abd5b390f322245e62982c9 Mon Sep 17 00:00:00 2001 From: joshvanl Date: Sat, 19 Oct 2024 03:14:11 +0100 Subject: [PATCH 16/27] Actors: Reminder migration state->scheduler Adds a backwards compatible migration mechanism to migrate Actor State Reminders to the new Scheduler based reminder system. Is activated on every Placement dissemination event, where all stored Actor reminders for the locally implemented Actor types are migrated to Scheduler Actor Reminer Jobs. Migration of reminders are implemented on a per daprd basis, whereby upon dissemination, each daprd will list the reminders in the state store and Scheduler for actor types that it implements. Reminders for actor IDs which are not hosted for that daprd are discarded preventing multiple daprd's from migrating the same reminder. Daprd will determine which reminders to migrate based on missing items or differing contents from state store and Scheduler. Signed-off-by: joshvanl --- dapr/proto/scheduler/v1/scheduler.proto | 5 +- pkg/actors/actors.go | 21 +- pkg/actors/actors_test.go | 5 + pkg/actors/internal/fake/reminders.go | 152 +++++++ pkg/actors/internal/fake/reminders_test.go | 24 + pkg/actors/internal/reminders.go | 3 + pkg/actors/internal/requests.go | 4 + pkg/actors/internal_actor_test.go | 2 + pkg/actors/reminders/migration/scheduler.go | 124 ++++++ .../reminders/migration/scheduler_test.go | 411 ++++++++++++++++++ pkg/actors/reminders/scheduler.go | 92 +++- pkg/actors/reminders/statestore.go | 24 + pkg/proto/scheduler/v1/scheduler.pb.go | 154 ++++--- pkg/runtime/runtime.go | 1 + .../wfengine/backends/actors/backend_test.go | 2 + pkg/runtime/wfengine/wfengine_test.go | 3 + pkg/scheduler/server/api.go | 9 +- .../server/internal/serialize/names.go | 43 ++ .../server/internal/serialize/names_test.go | 138 ++++++ .../framework/process/daprd/actors/actors.go | 135 ++++++ .../framework/process/daprd/actors/options.go | 84 ++++ .../process/daprd/workflow/options.go | 51 +++ .../process/daprd/workflow/workflow.go | 77 ++++ .../framework/process/placement/placement.go | 28 +- .../framework/process/scheduler/scheduler.go | 24 +- .../framework/process/sqlite/actors.go | 83 ++++ .../framework/process/sqlite/options.go | 26 +- .../framework/process/sqlite/sqlite.go | 53 ++- .../suite/actors/reminders/data.go | 77 ++++ .../actors/reminders/migration/migration.go | 18 + .../reminders/migration/scheduler/basic.go | 106 +++++ .../reminders/migration/scheduler/data.go | 122 ++++++ .../migration/scheduler/distributed.go | 164 +++++++ .../reminders/migration/scheduler/noop.go | 105 +++++ .../migration/scheduler/overwrite.go | 216 +++++++++ .../migration/scheduler/rebalance.go | 105 +++++ .../suite/actors/reminders/reminders.go | 1 + .../suite/actors/reminders/scheduler/data.go | 78 ++++ .../actors/reminders/serialization/common.go | 8 +- .../actors/reminders/serialization/default.go | 2 +- .../actors/reminders/serialization/json.go | 2 +- .../reminders/serialization/protobuf.go | 3 +- .../daprd/metrics/http/pathmatching/high.go | 16 +- 43 files changed, 2644 insertions(+), 157 deletions(-) create mode 100644 pkg/actors/internal/fake/reminders.go create mode 100644 pkg/actors/internal/fake/reminders_test.go create mode 100644 pkg/actors/reminders/migration/scheduler.go create mode 100644 pkg/actors/reminders/migration/scheduler_test.go create mode 100644 pkg/scheduler/server/internal/serialize/names_test.go create mode 100644 tests/integration/framework/process/daprd/actors/actors.go create mode 100644 tests/integration/framework/process/daprd/actors/options.go create mode 100644 tests/integration/framework/process/daprd/workflow/options.go create mode 100644 tests/integration/framework/process/daprd/workflow/workflow.go create mode 100644 tests/integration/framework/process/sqlite/actors.go create mode 100644 tests/integration/suite/actors/reminders/data.go create mode 100644 tests/integration/suite/actors/reminders/migration/migration.go create mode 100644 tests/integration/suite/actors/reminders/migration/scheduler/basic.go create mode 100644 tests/integration/suite/actors/reminders/migration/scheduler/data.go create mode 100644 tests/integration/suite/actors/reminders/migration/scheduler/distributed.go create mode 100644 tests/integration/suite/actors/reminders/migration/scheduler/noop.go create mode 100644 tests/integration/suite/actors/reminders/migration/scheduler/overwrite.go create mode 100644 tests/integration/suite/actors/reminders/migration/scheduler/rebalance.go create mode 100644 tests/integration/suite/actors/reminders/scheduler/data.go diff --git a/dapr/proto/scheduler/v1/scheduler.proto b/dapr/proto/scheduler/v1/scheduler.proto index 515315589d3..5be5fa1e8b0 100644 --- a/dapr/proto/scheduler/v1/scheduler.proto +++ b/dapr/proto/scheduler/v1/scheduler.proto @@ -165,8 +165,11 @@ message NamedJob { // name is the name of the job. string name = 1; + // The metadata associated with the job. + JobMetadata metadata = 2; + // The job scheduled. - Job job = 2; + Job job = 3; } // ListJobsRequest is the message used by the daprd sidecar to list all jobs. diff --git a/pkg/actors/actors.go b/pkg/actors/actors.go index 5fe97d8da0a..d1391a168e2 100644 --- a/pkg/actors/actors.go +++ b/pkg/actors/actors.go @@ -45,6 +45,7 @@ import ( "github.com/dapr/dapr/pkg/config" diag "github.com/dapr/dapr/pkg/diagnostics" diagUtils "github.com/dapr/dapr/pkg/diagnostics/utils" + "github.com/dapr/dapr/pkg/healthz" invokev1 "github.com/dapr/dapr/pkg/messaging/v1" "github.com/dapr/dapr/pkg/modes" commonv1pb "github.com/dapr/dapr/pkg/proto/common/v1" @@ -144,6 +145,7 @@ type actorsRuntime struct { closed atomic.Bool closeCh chan struct{} apiLevel atomic.Uint32 + htarget healthz.Target lock sync.Mutex internalReminderInProgress map[string]struct{} @@ -165,6 +167,7 @@ type ActorsOpts struct { Security security.Handler SchedulerClients *clients.Clients SchedulerReminders bool + Healthz healthz.Healthz // TODO: @joshvanl Remove in Dapr 1.12 when ActorStateTTL is finalized. StateTTLEnabled bool @@ -194,6 +197,7 @@ func newActorsWithClock(opts ActorsOpts, clock clock.WithTicker) (ActorRuntime, internalActors: haxmap.New[string, InternalActor](32), compStore: opts.CompStore, sec: opts.Security, + htarget: opts.Healthz.AddTarget(), internalReminderInProgress: map[string]struct{}{}, schedulerReminderFeatureEnabled: opts.SchedulerReminders, @@ -242,9 +246,11 @@ func newActorsWithClock(opts ActorsOpts, clock clock.WithTicker) (ActorRuntime, } log.Debug("Using Scheduler service for reminders.") a.actorsReminders = reminders.NewScheduler(reminders.SchedulerOptions{ - Clients: opts.Config.SchedulerClients, - Namespace: opts.Config.Namespace, - AppID: opts.Config.AppID, + Clients: opts.Config.SchedulerClients, + Namespace: opts.Config.Namespace, + AppID: opts.Config.AppID, + ProviderOpts: providerOpts, + ListActorTypesFn: a.Entities, }) } else { factory, err := opts.Config.GetRemindersProvider(a.placement) @@ -252,12 +258,12 @@ func newActorsWithClock(opts ActorsOpts, clock clock.WithTicker) (ActorRuntime, return nil, fmt.Errorf("failed to initialize reminders provider: %w", err) } a.actorsReminders = factory(providerOpts) - - a.actorsReminders.SetExecuteReminderFn(a.executeReminder) - a.actorsReminders.SetStateStoreProviderFn(a.stateStore) - a.actorsReminders.SetLookupActorFn(a.isActorLocallyHosted) } + a.actorsReminders.SetExecuteReminderFn(a.executeReminder) + a.actorsReminders.SetStateStoreProviderFn(a.stateStore) + a.actorsReminders.SetLookupActorFn(a.isActorLocallyHosted) + a.idleActorProcessor = eventqueue.NewProcessor[string, *actor](a.idleProcessorExecuteFn).WithClock(clock) return a, nil } @@ -317,6 +323,7 @@ func (a *actorsRuntime) Init(ctx context.Context) (err error) { a.placement.SetOnTableUpdateFn(func() { a.drainRebalancedActors() a.actorsReminders.OnPlacementTablesUpdated(ctx) + a.htarget.Ready() }) a.checker, err = a.getAppHealthChecker() diff --git a/pkg/actors/actors_test.go b/pkg/actors/actors_test.go index a2c35f34afc..404d853b9a9 100644 --- a/pkg/actors/actors_test.go +++ b/pkg/actors/actors_test.go @@ -41,6 +41,7 @@ import ( "github.com/dapr/dapr/pkg/apis/resiliency/v1alpha1" "github.com/dapr/dapr/pkg/channel" "github.com/dapr/dapr/pkg/config" + "github.com/dapr/dapr/pkg/healthz" invokev1 "github.com/dapr/dapr/pkg/messaging/v1" "github.com/dapr/dapr/pkg/modes" commonv1pb "github.com/dapr/dapr/pkg/proto/common/v1" @@ -207,6 +208,7 @@ func (b *runtimeBuilder) buildActorRuntime(t *testing.T) *actorsRuntime { TracingSpec: config.TracingSpec{SamplingRate: "1"}, Resiliency: resiliency.FromConfigurations(log, testResiliency), StateStoreName: storeName, + Healthz: healthz.New(), }, clock) require.NoError(t, err) @@ -235,6 +237,7 @@ func newTestActorsRuntimeWithMock(t *testing.T, appChannel channel.AppChannel) * TracingSpec: config.TracingSpec{SamplingRate: "1"}, Resiliency: resiliency.New(log), StateStoreName: "actorStore", + Healthz: healthz.New(), MockPlacement: NewMockPlacement(TestAppID), }, clock) require.NoError(t, err) @@ -258,6 +261,7 @@ func newTestActorsRuntimeWithMockWithoutPlacement(t *testing.T, appChannel chann TracingSpec: config.TracingSpec{SamplingRate: "1"}, Resiliency: resiliency.New(log), StateStoreName: "actorStore", + Healthz: healthz.New(), }, clock) require.NoError(t, err) @@ -280,6 +284,7 @@ func newTestActorsRuntimeWithMockAndNoStore(t *testing.T, appChannel channel.App TracingSpec: config.TracingSpec{SamplingRate: "1"}, Resiliency: resiliency.New(log), StateStoreName: "actorStore", + Healthz: healthz.New(), }, clock) require.NoError(t, err) diff --git a/pkg/actors/internal/fake/reminders.go b/pkg/actors/internal/fake/reminders.go new file mode 100644 index 00000000000..06ed5e9487b --- /dev/null +++ b/pkg/actors/internal/fake/reminders.go @@ -0,0 +1,152 @@ +/* +Copyright 2024 The Dapr 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 fake + +import ( + "context" + + "github.com/dapr/dapr/pkg/actors/internal" +) + +type FakeRemindersProvider struct { + initFn func(ctx context.Context) error + getReminderFn func(ctx context.Context, req *internal.GetReminderRequest) (*internal.Reminder, error) + createReminderFn func(ctx context.Context, req *internal.CreateReminderRequest) error + deleteReminderFn func(ctx context.Context, req internal.DeleteReminderRequest) error + listRemindersFn func(ctx context.Context, req internal.ListRemindersRequest) ([]*internal.Reminder, error) + drainRebalancedRemindersFn func(actorType string, actorID string) + onPlacementTablesUpdatedFn func(ctx context.Context) + setExecuteReminderFn func(fn internal.ExecuteReminderFn) + setStateStoreProviderFn func(fn internal.StateStoreProviderFn) + setLookupActorFn func(fn internal.LookupActorFn) +} + +func NewRemindersProvider() *FakeRemindersProvider { + return &FakeRemindersProvider{ + initFn: func(ctx context.Context) error { + return nil + }, + getReminderFn: func(ctx context.Context, req *internal.GetReminderRequest) (*internal.Reminder, error) { + return nil, nil + }, + createReminderFn: func(ctx context.Context, req *internal.CreateReminderRequest) error { + return nil + }, + deleteReminderFn: func(ctx context.Context, req internal.DeleteReminderRequest) error { + return nil + }, + listRemindersFn: func(ctx context.Context, req internal.ListRemindersRequest) ([]*internal.Reminder, error) { + return nil, nil + }, + drainRebalancedRemindersFn: func(actorType string, actorID string) {}, + onPlacementTablesUpdatedFn: func(ctx context.Context) {}, + setExecuteReminderFn: func(fn internal.ExecuteReminderFn) {}, + setStateStoreProviderFn: func(fn internal.StateStoreProviderFn) {}, + setLookupActorFn: func(fn internal.LookupActorFn) {}, + } +} + +func (f *FakeRemindersProvider) WithInit(fn func(ctx context.Context) error) *FakeRemindersProvider { + f.initFn = fn + return f +} + +func (f *FakeRemindersProvider) WithGetReminder(fn func(ctx context.Context, req *internal.GetReminderRequest) (*internal.Reminder, error)) *FakeRemindersProvider { + f.getReminderFn = fn + return f +} + +func (f *FakeRemindersProvider) WithCreateReminder(fn func(ctx context.Context, req *internal.CreateReminderRequest) error) *FakeRemindersProvider { + f.createReminderFn = fn + return f +} + +func (f *FakeRemindersProvider) WithDeleteReminder(fn func(ctx context.Context, req internal.DeleteReminderRequest) error) *FakeRemindersProvider { + f.deleteReminderFn = fn + return f +} + +func (f *FakeRemindersProvider) WithListReminders(fn func(ctx context.Context, req internal.ListRemindersRequest) ([]*internal.Reminder, error)) *FakeRemindersProvider { + f.listRemindersFn = fn + return f +} + +func (f *FakeRemindersProvider) WithDrainRebalancedReminders(fn func(actorType string, actorID string)) *FakeRemindersProvider { + f.drainRebalancedRemindersFn = fn + return f +} + +func (f *FakeRemindersProvider) WithOnPlacementTablesUpdated(fn func(ctx context.Context)) *FakeRemindersProvider { + f.onPlacementTablesUpdatedFn = fn + return f +} + +func (f *FakeRemindersProvider) WithSetExecuteReminder(fn func(fn internal.ExecuteReminderFn)) *FakeRemindersProvider { + f.setExecuteReminderFn = fn + return f +} + +func (f *FakeRemindersProvider) WithSetStateStoreProvider(fn func(fn internal.StateStoreProviderFn)) *FakeRemindersProvider { + f.setStateStoreProviderFn = fn + return f +} + +func (f *FakeRemindersProvider) WithSetLookupActor(fn func(fn internal.LookupActorFn)) *FakeRemindersProvider { + f.setLookupActorFn = fn + return f +} + +func (f *FakeRemindersProvider) Init(ctx context.Context) error { + return f.initFn(ctx) +} + +func (f *FakeRemindersProvider) GetReminder(ctx context.Context, req *internal.GetReminderRequest) (*internal.Reminder, error) { + return f.getReminderFn(ctx, req) +} + +func (f *FakeRemindersProvider) CreateReminder(ctx context.Context, req *internal.CreateReminderRequest) error { + return f.createReminderFn(ctx, req) +} + +func (f *FakeRemindersProvider) DeleteReminder(ctx context.Context, req internal.DeleteReminderRequest) error { + return f.deleteReminderFn(ctx, req) +} + +func (f *FakeRemindersProvider) ListReminders(ctx context.Context, req internal.ListRemindersRequest) ([]*internal.Reminder, error) { + return f.listRemindersFn(ctx, req) +} + +func (f *FakeRemindersProvider) DrainRebalancedReminders(actorType string, actorID string) { + f.drainRebalancedRemindersFn(actorType, actorID) +} + +func (f *FakeRemindersProvider) OnPlacementTablesUpdated(ctx context.Context) { + f.onPlacementTablesUpdatedFn(ctx) +} + +func (f *FakeRemindersProvider) SetExecuteReminderFn(fn internal.ExecuteReminderFn) { + f.setExecuteReminderFn(fn) +} + +func (f *FakeRemindersProvider) SetStateStoreProviderFn(fn internal.StateStoreProviderFn) { + f.setStateStoreProviderFn(fn) +} + +func (f *FakeRemindersProvider) SetLookupActorFn(fn internal.LookupActorFn) { + f.setLookupActorFn(fn) +} + +func (f *FakeRemindersProvider) Close() error { + return nil +} diff --git a/pkg/actors/internal/fake/reminders_test.go b/pkg/actors/internal/fake/reminders_test.go new file mode 100644 index 00000000000..83e64a0534e --- /dev/null +++ b/pkg/actors/internal/fake/reminders_test.go @@ -0,0 +1,24 @@ +/* +Copyright 2024 The Dapr 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 fake + +import ( + "testing" + + "github.com/dapr/dapr/pkg/actors/internal" +) + +func Test_FakeRemindersProvider(t *testing.T) { + var _ internal.RemindersProvider = NewRemindersProvider() +} diff --git a/pkg/actors/internal/reminders.go b/pkg/actors/internal/reminders.go index 51490ee9e3b..586572e2f22 100644 --- a/pkg/actors/internal/reminders.go +++ b/pkg/actors/internal/reminders.go @@ -33,6 +33,8 @@ type LookupActorFn func(ctx context.Context, actorType string, actorID string) ( type StateStoreProviderFn func() (string, TransactionalStateStore, error) // RemindersProvider is the interface for the object that provides reminders services. +// +//nolint:interfacebloat type RemindersProvider interface { io.Closer @@ -40,6 +42,7 @@ type RemindersProvider interface { GetReminder(ctx context.Context, req *GetReminderRequest) (*Reminder, error) CreateReminder(ctx context.Context, req *CreateReminderRequest) error DeleteReminder(ctx context.Context, req DeleteReminderRequest) error + ListReminders(ctx context.Context, req ListRemindersRequest) ([]*Reminder, error) DrainRebalancedReminders(actorType string, actorID string) OnPlacementTablesUpdated(ctx context.Context) diff --git a/pkg/actors/internal/requests.go b/pkg/actors/internal/requests.go index 4b3a23ba8fa..c7eaa845eff 100644 --- a/pkg/actors/internal/requests.go +++ b/pkg/actors/internal/requests.go @@ -182,6 +182,10 @@ func (req DeleteReminderRequest) Key() string { return req.ActorType + daprSeparator + req.ActorID + daprSeparator + req.Name } +type ListRemindersRequest struct { + ActorType string +} + // DeleteTimerRequest is a request object for deleting a timer. type DeleteTimerRequest struct { Name string diff --git a/pkg/actors/internal_actor_test.go b/pkg/actors/internal_actor_test.go index d49fed43a65..4bc16dfdccd 100644 --- a/pkg/actors/internal_actor_test.go +++ b/pkg/actors/internal_actor_test.go @@ -28,6 +28,7 @@ import ( "github.com/dapr/dapr/pkg/actors/internal" "github.com/dapr/dapr/pkg/config" + "github.com/dapr/dapr/pkg/healthz" invokev1 "github.com/dapr/dapr/pkg/messaging/v1" "github.com/dapr/dapr/pkg/proto/internals/v1" "github.com/dapr/dapr/pkg/resiliency" @@ -108,6 +109,7 @@ func newTestActorsRuntimeWithInternalActors(internalActors map[string]InternalAc StateStoreName: "actorStore", Security: fake.New(), MockPlacement: NewMockPlacement(TestAppID), + Healthz: healthz.New(), }) if err != nil { return nil, err diff --git a/pkg/actors/reminders/migration/scheduler.go b/pkg/actors/reminders/migration/scheduler.go new file mode 100644 index 00000000000..937411fdef8 --- /dev/null +++ b/pkg/actors/reminders/migration/scheduler.go @@ -0,0 +1,124 @@ +/* +Copyright 2024 The Dapr 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 migration + +import ( + "bytes" + "context" + "fmt" + "math" + "time" + + "k8s.io/utils/clock" + + "github.com/dapr/dapr/pkg/actors/internal" + "github.com/dapr/kit/logger" +) + +var log = logger.NewLogger("dapr.runtime.actors.reminders.migration") + +type ToSchedulerOptions struct { + Clock clock.Clock + ActorTypes []string + + LookUpActorFn internal.LookupActorFn + StateReminders internal.RemindersProvider + SchedulerReminders internal.RemindersProvider +} + +func ToScheduler(ctx context.Context, opts ToSchedulerOptions) error { + log.Infof("Running actor reminder migration from state store to scheduler") + + stateReminders := make(map[string][]*internal.Reminder) + schedulerReminders := make(map[string][]*internal.Reminder) + + for _, actorType := range opts.ActorTypes { + log.Debugf("Listing state reminders for actor type %s", actorType) + stateR, err := opts.StateReminders.ListReminders(ctx, internal.ListRemindersRequest{ + ActorType: actorType, + }) + if err != nil { + return err + } + for i := range stateR { + if ok, _ := opts.LookUpActorFn(ctx, actorType, stateR[i].ActorID); ok { + log.Debugf("Hosted state reminder %s for actor %s in state store", stateR[i].Key(), stateR[i].ActorID) + stateReminders[actorType] = append(stateReminders[actorType], stateR[i]) + } + } + + log.Debugf("Listing scheduler reminders for actor type %s", actorType) + schedR, err := opts.SchedulerReminders.ListReminders(ctx, internal.ListRemindersRequest{ + ActorType: actorType, + }) + if err != nil { + return err + } + schedulerReminders[actorType] = schedR + } + + var missingReminders []*internal.Reminder + for _, actorType := range opts.ActorTypes { + for _, stateReminder := range stateReminders[actorType] { + var exists bool + for _, schedulerReminder := range schedulerReminders[actorType] { + if stateReminder.ActorID == schedulerReminder.ActorID && + stateReminder.Name == schedulerReminder.Name { + exists = stateReminder.DueTime == schedulerReminder.DueTime && + stateReminder.Period.String() == schedulerReminder.Period.String() && + bytes.Equal(stateReminder.Data, schedulerReminder.Data) && + math.Abs(float64(stateReminder.ExpirationTime.Sub(schedulerReminder.ExpirationTime))) < float64(time.Minute) + + break + } + } + + if !exists { + log.Debugf("Found missing scheduler reminder %s", stateReminder.Key()) + missingReminders = append(missingReminders, stateReminder) + } + } + } + + if len(missingReminders) == 0 { + log.Infof("Skipping migration, no missing scheduler reminders found") + } + + log.Infof("Found %d missing scheduler reminders from state store", len(missingReminders)) + for _, missing := range missingReminders { + log.Infof("Creating missing scheduler reminder %s", missing.Key()) + + var ttl string + if !missing.ExpirationTime.IsZero() { + ttl = missing.ExpirationTime.UTC().Format(time.RFC3339) + } + + err := opts.SchedulerReminders.CreateReminder(ctx, &internal.CreateReminderRequest{ + Name: missing.Name, + ActorType: missing.ActorType, + ActorID: missing.ActorID, + Data: missing.Data, + DueTime: missing.DueTime, + Period: missing.Period.String(), + TTL: ttl, + }) + if err != nil { + return fmt.Errorf("failed to migrate reminder %s: %w", missing.Key(), err) + } + } + + log.Infof("Migrated %d reminders from state store to scheduler successfully", len(missingReminders)) + + return nil +} diff --git a/pkg/actors/reminders/migration/scheduler_test.go b/pkg/actors/reminders/migration/scheduler_test.go new file mode 100644 index 00000000000..9fd4effa851 --- /dev/null +++ b/pkg/actors/reminders/migration/scheduler_test.go @@ -0,0 +1,411 @@ +/* +Copyright 2024 The Dapr 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 migration + +import ( + "context" + "encoding/json" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/dapr/dapr/pkg/actors/internal" + "github.com/dapr/dapr/pkg/actors/internal/fake" +) + +func Test_ToScheduler(t *testing.T) { + reminderPeriod1, err := internal.NewReminderPeriod("PT1S") + require.NoError(t, err) + reminderPeriod2, err := internal.NewReminderPeriod("PT2S") + require.NoError(t, err) + + expTime1 := time.Now().Add(time.Second) + expTime2 := time.Now().Add(time.Minute * 2) + + tests := map[string]struct { + inputActorTypes []string + stateListReminders map[string][]*internal.Reminder + schedulerListReminders map[string][]*internal.Reminder + lookupActors map[string]map[string]bool + expCreateReminders []*internal.CreateReminderRequest + }{ + "no types": {}, + "no reminders": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": {}, + "type2": {}, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": {}, + "type2": {}, + }, + }, + "not hosted": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + }, + "type2": { + {ActorType: "type2", ActorID: "id2", Name: "name2", DueTime: "0s"}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": {}, + "type2": {}, + }, + lookupActors: map[string]map[string]bool{ + "type1": {"id1": false}, + "type2": {"id2": false}, + }, + }, + "missing": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + }, + "type2": { + {ActorType: "type2", ActorID: "id2", Name: "name2", DueTime: "0s"}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": {}, + "type2": {}, + }, + lookupActors: map[string]map[string]bool{ + "type1": {"id1": true}, + "type2": {"id2": true}, + }, + expCreateReminders: []*internal.CreateReminderRequest{ + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name2", DueTime: "0s"}, + }, + }, + "missing multiple ids": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id2", Name: "name2", DueTime: "0s"}, + }, + "type2": { + {ActorType: "type2", ActorID: "id3", Name: "name3", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id4", Name: "name4", DueTime: "0s"}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": {}, + "type2": {}, + }, + lookupActors: map[string]map[string]bool{ + "type1": {"id1": true, "id2": true}, + "type2": {"id3": true, "id4": true}, + }, + expCreateReminders: []*internal.CreateReminderRequest{ + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id2", Name: "name2", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id3", Name: "name3", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id4", Name: "name4", DueTime: "0s"}, + }, + }, + "some not hosted": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id2", Name: "name2", DueTime: "0s"}, + }, + "type2": { + {ActorType: "type2", ActorID: "id3", Name: "name3", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id4", Name: "name4", DueTime: "0s"}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": {}, + "type2": {}, + }, + lookupActors: map[string]map[string]bool{ + "type1": {"id1": true, "id2": false}, + "type2": {"id3": true, "id4": false}, + }, + expCreateReminders: []*internal.CreateReminderRequest{ + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id3", Name: "name3", DueTime: "0s"}, + }, + }, + "multiple reminders per actor": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id2", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id2", Name: "name2", DueTime: "0s"}, + }, + "type2": { + {ActorType: "type2", ActorID: "id2", Name: "name3", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name4", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name5", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name6", DueTime: "0s"}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": {}, + "type2": {}, + }, + lookupActors: map[string]map[string]bool{ + "type1": {"id1": true, "id2": false}, + "type2": {"id2": true}, + }, + expCreateReminders: []*internal.CreateReminderRequest{ + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name3", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name4", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name5", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name6", DueTime: "0s"}, + }, + }, + "reminders already migrated": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id2", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id2", Name: "name2", DueTime: "0s"}, + }, + "type2": { + {ActorType: "type2", ActorID: "id2", Name: "name3", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name4", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name5", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name6", DueTime: "0s"}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id2", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id2", Name: "name2", DueTime: "0s"}, + }, + "type2": { + {ActorType: "type2", ActorID: "id2", Name: "name3", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name4", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name5", DueTime: "0s"}, + {ActorType: "type2", ActorID: "id2", Name: "name6", DueTime: "0s"}, + }, + }, + lookupActors: map[string]map[string]bool{ + "type1": {"id1": true, "id2": false}, + "type2": {"id2": true}, + }, + }, + "reminders exist but wrong due time": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "1s"}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "1s"}, + }, + "type2": {}, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s"}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s"}, + }, + "type2": {}, + }, + lookupActors: map[string]map[string]bool{ + "type1": {"id1": true}, + }, + expCreateReminders: []*internal.CreateReminderRequest{ + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "1s"}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "1s"}, + }, + }, + "reminders exist with same data": { + inputActorTypes: []string{"type1"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Data: json.RawMessage("data1")}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Data: json.RawMessage("data2")}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Data: json.RawMessage("data1")}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Data: json.RawMessage("data2")}, + }, + }, + lookupActors: map[string]map[string]bool{ + "type1": {"id1": true}, + }, + }, + "reminders exist but wrong data": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Data: json.RawMessage("data1")}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Data: json.RawMessage("data2")}, + }, + "type2": {}, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Data: json.RawMessage("data2")}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Data: json.RawMessage("data1")}, + }, + "type2": {}, + }, + lookupActors: map[string]map[string]bool{ + "type1": {"id1": true}, + "type2": {"id2": true}, + }, + expCreateReminders: []*internal.CreateReminderRequest{ + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Data: json.RawMessage("data1")}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Data: json.RawMessage("data2")}, + }, + }, + "remiders exist with same period": { + inputActorTypes: []string{"type1", "type2"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Period: reminderPeriod1}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Period: reminderPeriod1}, + }, + "type2": {}, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Period: reminderPeriod1}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Period: reminderPeriod1}, + }, + "type2": {}, + }, + lookupActors: map[string]map[string]bool{"type1": {"id1": true}}, + }, + "reminders exist but wrong period": { + inputActorTypes: []string{"type1"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Period: reminderPeriod1}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Period: reminderPeriod1}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Period: reminderPeriod2}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Period: reminderPeriod2}, + }, + }, + lookupActors: map[string]map[string]bool{"type1": {"id1": true}}, + expCreateReminders: []*internal.CreateReminderRequest{ + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", Period: "PT1S"}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", Period: "PT1S"}, + }, + }, + "reminders exist with same ttl": { + inputActorTypes: []string{"type1"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", ExpirationTime: expTime1}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", ExpirationTime: expTime1}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", ExpirationTime: expTime1}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", ExpirationTime: expTime1}, + }, + }, + lookupActors: map[string]map[string]bool{"type1": {"id1": true}}, + expCreateReminders: []*internal.CreateReminderRequest{}, + }, + "reminders exist but wrong ttl": { + inputActorTypes: []string{"type1"}, + stateListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", ExpirationTime: expTime1}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", ExpirationTime: expTime1}, + }, + }, + schedulerListReminders: map[string][]*internal.Reminder{ + "type1": { + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", ExpirationTime: expTime2}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", ExpirationTime: expTime2}, + }, + }, + expCreateReminders: []*internal.CreateReminderRequest{ + {ActorType: "type1", ActorID: "id1", Name: "name1", DueTime: "0s", TTL: expTime1.UTC().Format(time.RFC3339)}, + {ActorType: "type1", ActorID: "id1", Name: "name2", DueTime: "0s", TTL: expTime1.UTC().Format(time.RFC3339)}, + }, + lookupActors: map[string]map[string]bool{"type1": {"id1": true}}, + }, + } + + for name, test := range tests { + t.Run(name, func(t *testing.T) { + t.Parallel() + + stateStore := fake.NewRemindersProvider(). + WithListReminders(func(_ context.Context, req internal.ListRemindersRequest) ([]*internal.Reminder, error) { + assert.Contains(t, test.stateListReminders, req.ActorType) + rs, ok := test.stateListReminders[req.ActorType] + require.True(t, ok) + return rs, nil + }). + WithCreateReminder(func(context.Context, *internal.CreateReminderRequest) error { + t.Fatalf("unexpected call to CreateReminders") + return nil + }) + + var creates int + t.Cleanup(func() { assert.Len(t, test.expCreateReminders, creates) }) + + scheduler := fake.NewRemindersProvider(). + WithListReminders(func(_ context.Context, req internal.ListRemindersRequest) ([]*internal.Reminder, error) { + assert.Contains(t, test.schedulerListReminders, req.ActorType) + rs, ok := test.schedulerListReminders[req.ActorType] + require.True(t, ok) + return rs, nil + }). + WithCreateReminder(func(_ context.Context, req *internal.CreateReminderRequest) error { + assert.Contains(t, test.expCreateReminders, req) + creates++ + return nil + }) + + lookupFn := func(_ context.Context, actorType string, actorID string) (bool, string) { + ids, ok := test.lookupActors[actorType] + require.True(t, ok) + local, ok := ids[actorID] + require.True(t, ok) + return local, "" + } + + require.NoError(t, ToScheduler(context.Background(), ToSchedulerOptions{ + ActorTypes: test.inputActorTypes, + LookUpActorFn: lookupFn, + StateReminders: stateStore, + SchedulerReminders: scheduler, + })) + }) + } +} diff --git a/pkg/actors/reminders/scheduler.go b/pkg/actors/reminders/scheduler.go index 47ebf0438ca..98d6abc7286 100644 --- a/pkg/actors/reminders/scheduler.go +++ b/pkg/actors/reminders/scheduler.go @@ -24,6 +24,7 @@ import ( "google.golang.org/protobuf/types/known/wrapperspb" "github.com/dapr/dapr/pkg/actors/internal" + "github.com/dapr/dapr/pkg/actors/reminders/migration" apierrors "github.com/dapr/dapr/pkg/api/errors" schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" "github.com/dapr/dapr/pkg/runtime/scheduler/clients" @@ -32,23 +33,30 @@ import ( ) type SchedulerOptions struct { - Namespace string - AppID string - Clients *clients.Clients + Namespace string + AppID string + Clients *clients.Clients + ProviderOpts internal.ActorsProviderOptions + ListActorTypesFn func() []string } // Implements a reminders provider that does nothing when using Scheduler Service. type scheduler struct { - namespace string - appID string - clients *clients.Clients + namespace string + appID string + clients *clients.Clients + lookUpActorFn internal.LookupActorFn + stateReminder internal.RemindersProvider + listActorTypesFn func() []string } func NewScheduler(opts SchedulerOptions) internal.RemindersProvider { return &scheduler{ - clients: opts.Clients, - namespace: opts.Namespace, - appID: opts.AppID, + clients: opts.Clients, + namespace: opts.Namespace, + appID: opts.AppID, + stateReminder: NewStateStore(opts.ProviderOpts), + listActorTypesFn: opts.ListActorTypesFn, } } @@ -56,9 +64,12 @@ func (s *scheduler) SetExecuteReminderFn(fn internal.ExecuteReminderFn) { } func (s *scheduler) SetStateStoreProviderFn(fn internal.StateStoreProviderFn) { + s.stateReminder.SetStateStoreProviderFn(fn) } func (s *scheduler) SetLookupActorFn(fn internal.LookupActorFn) { + s.lookUpActorFn = fn + s.stateReminder.SetLookupActorFn(fn) } func (s *scheduler) SetMetricsCollectorFn(fn remindersMetricsCollectorFn) { @@ -66,6 +77,15 @@ func (s *scheduler) SetMetricsCollectorFn(fn remindersMetricsCollectorFn) { // OnPlacementTablesUpdated is invoked when the actors runtime received an updated placement tables. func (s *scheduler) OnPlacementTablesUpdated(ctx context.Context) { + err := migration.ToScheduler(ctx, migration.ToSchedulerOptions{ + ActorTypes: s.listActorTypesFn(), + LookUpActorFn: s.lookUpActorFn, + StateReminders: s.stateReminder, + SchedulerReminders: s, + }) + if err != nil { + log.Errorf("Error attempting to migrate reminders to scheduler: %s", err) + } } func (s *scheduler) DrainRebalancedReminders(actorType string, actorID string) { @@ -204,15 +224,10 @@ func (s *scheduler) GetReminder(ctx context.Context, req *internal.GetReminderRe return nil, err } - var data json.RawMessage - if err := json.Unmarshal(jsonBytes, &data); err != nil { - return nil, err - } - reminder := &internal.Reminder{ ActorID: req.ActorID, ActorType: req.ActorType, - Data: data, + Data: jsonBytes, Period: internal.NewSchedulerReminderPeriod(job.GetJob().GetSchedule(), job.GetJob().GetRepeats()), DueTime: job.GetJob().GetDueTime(), } @@ -248,3 +263,50 @@ func (s *scheduler) DeleteReminder(ctx context.Context, req internal.DeleteRemin } return err } + +func (s *scheduler) ListReminders(ctx context.Context, req internal.ListRemindersRequest) ([]*internal.Reminder, error) { + client, err := s.clients.Next(ctx) + if err != nil { + return nil, err + } + resp, err := client.ListJobs(ctx, &schedulerv1pb.ListJobsRequest{ + Metadata: &schedulerv1pb.JobMetadata{ + AppId: s.appID, + Namespace: s.namespace, + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Actor{ + Actor: &schedulerv1pb.TargetActorReminder{ + Type: req.ActorType, + }, + }, + }, + }, + }) + if err != nil { + return nil, err + } + reminders := make([]*internal.Reminder, len(resp.GetJobs())) + for i, named := range resp.GetJobs() { + actor := named.GetMetadata().GetTarget().GetActor() + if actor == nil { + log.Warnf("Skipping reminder job %s with unsupported target type %s", named.GetName(), named.GetMetadata().GetTarget().String()) + continue + } + + job := named.GetJob() + jsonBytes, err := protojson.Marshal(job.GetData()) + if err != nil { + return nil, err + } + + reminders[i] = &internal.Reminder{ + Name: named.GetName(), + ActorID: actor.GetId(), + ActorType: actor.GetType(), + Data: jsonBytes, + Period: internal.NewSchedulerReminderPeriod(job.GetSchedule(), job.GetRepeats()), + DueTime: job.GetDueTime(), + } + } + return reminders, nil +} diff --git a/pkg/actors/reminders/statestore.go b/pkg/actors/reminders/statestore.go index 3dcd31016a7..e103dfd7dee 100644 --- a/pkg/actors/reminders/statestore.go +++ b/pkg/actors/reminders/statestore.go @@ -238,6 +238,30 @@ func (r *statestore) GetReminder(ctx context.Context, req *internal.GetReminderR return nil, nil } +func (r *statestore) ListReminders(ctx context.Context, req internal.ListRemindersRequest) ([]*internal.Reminder, error) { + list, _, err := r.getRemindersForActorType(ctx, req.ActorType, false) + if err != nil { + return nil, err + } + + reminders := make([]*internal.Reminder, len(list)) + for i, r := range list { + reminders[i] = &internal.Reminder{ + Name: r.Reminder.Name, + ActorID: r.Reminder.ActorID, + ActorType: r.Reminder.ActorType, + Data: r.Reminder.Data, + DueTime: r.Reminder.DueTime, + Period: r.Reminder.Period, + RegisteredTime: r.Reminder.RegisteredTime, + ExpirationTime: r.Reminder.ExpirationTime, + Callback: r.Reminder.Callback, + } + } + + return reminders, nil +} + func (r *statestore) DeleteReminder(ctx context.Context, req internal.DeleteReminderRequest) error { if !r.waitForEvaluationChan() { return errors.New("error deleting reminder: timed out after 30s") diff --git a/pkg/proto/scheduler/v1/scheduler.pb.go b/pkg/proto/scheduler/v1/scheduler.pb.go index dcc823268b4..95be275ade6 100644 --- a/pkg/proto/scheduler/v1/scheduler.pb.go +++ b/pkg/proto/scheduler/v1/scheduler.pb.go @@ -947,8 +947,10 @@ type NamedJob struct { // name is the name of the job. Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // The metadata associated with the job. + Metadata *JobMetadata `protobuf:"bytes,2,opt,name=metadata,proto3" json:"metadata,omitempty"` // The job scheduled. - Job *Job `protobuf:"bytes,2,opt,name=job,proto3" json:"job,omitempty"` + Job *Job `protobuf:"bytes,3,opt,name=job,proto3" json:"job,omitempty"` } func (x *NamedJob) Reset() { @@ -990,6 +992,13 @@ func (x *NamedJob) GetName() string { return "" } +func (x *NamedJob) GetMetadata() *JobMetadata { + if x != nil { + return x.Metadata + } + return nil +} + func (x *NamedJob) GetJob() *Job { if x != nil { return x.Job @@ -1202,59 +1211,63 @@ var file_dapr_proto_scheduler_v1_scheduler_proto_rawDesc = []byte{ 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0x13, 0x0a, 0x11, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4e, 0x0a, 0x08, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x4a, - 0x6f, 0x62, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2e, 0x0a, 0x03, 0x6a, 0x6f, 0x62, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, - 0x62, 0x52, 0x03, 0x6a, 0x6f, 0x62, 0x22, 0x53, 0x0a, 0x0f, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, - 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x6d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x64, 0x61, - 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, - 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0x49, 0x0a, 0x10, 0x4c, - 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x35, 0x0a, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, - 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, - 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x4a, 0x6f, 0x62, - 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x32, 0x87, 0x04, 0x0a, 0x09, 0x53, 0x63, 0x68, 0x65, 0x64, - 0x75, 0x6c, 0x65, 0x72, 0x12, 0x6a, 0x0a, 0x0b, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, - 0x4a, 0x6f, 0x62, 0x12, 0x2b, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x90, 0x01, 0x0a, 0x08, 0x4e, 0x61, 0x6d, 0x65, 0x64, + 0x4a, 0x6f, 0x62, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x40, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x64, 0x61, 0x70, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, + 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, + 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x2e, 0x0a, 0x03, 0x6a, 0x6f, 0x62, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, + 0x2e, 0x4a, 0x6f, 0x62, 0x52, 0x03, 0x6a, 0x6f, 0x62, 0x22, 0x53, 0x0a, 0x0f, 0x4c, 0x69, 0x73, + 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, + 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, + 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0x49, + 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x35, 0x0a, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x21, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x64, + 0x4a, 0x6f, 0x62, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x32, 0x87, 0x04, 0x0a, 0x09, 0x53, 0x63, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x6a, 0x0a, 0x0b, 0x53, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x4a, 0x6f, 0x62, 0x12, 0x2b, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, + 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, - 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x2c, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, - 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, - 0x75, 0x6c, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x5b, 0x0a, 0x06, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x12, 0x26, 0x2e, 0x64, 0x61, 0x70, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, - 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, - 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, - 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x12, 0x29, 0x2e, 0x64, 0x61, 0x70, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, - 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x06, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x12, 0x26, 0x2e, + 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x68, 0x0a, 0x09, 0x57, 0x61, 0x74, 0x63, 0x68, 0x4a, 0x6f, 0x62, 0x73, - 0x12, 0x29, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, - 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, - 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x64, 0x61, - 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, - 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x4a, 0x6f, 0x62, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x61, 0x0a, - 0x08, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x28, 0x2e, 0x64, 0x61, 0x70, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, - 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x42, 0x37, 0x5a, 0x35, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x64, - 0x61, 0x70, 0x72, 0x2f, 0x64, 0x61, 0x70, 0x72, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2f, 0x76, 0x31, 0x3b, - 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x64, 0x0a, 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x12, 0x29, 0x2e, + 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, + 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, + 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, + 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x68, 0x0a, 0x09, 0x57, 0x61, 0x74, 0x63, 0x68, 0x4a, + 0x6f, 0x62, 0x73, 0x12, 0x29, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x61, + 0x74, 0x63, 0x68, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, + 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x4a, 0x6f, + 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, + 0x12, 0x61, 0x0a, 0x08, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x28, 0x2e, 0x64, + 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, + 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x64, 0x61, 0x70, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2e, 0x76, 0x31, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x42, 0x37, 0x5a, 0x35, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, + 0x6d, 0x2f, 0x64, 0x61, 0x70, 0x72, 0x2f, 0x64, 0x61, 0x70, 0x72, 0x2f, 0x70, 0x6b, 0x67, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x2f, + 0x76, 0x31, 0x3b, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1305,24 +1318,25 @@ var file_dapr_proto_scheduler_v1_scheduler_proto_depIdxs = []int32{ 4, // 10: dapr.proto.scheduler.v1.GetJobRequest.metadata:type_name -> dapr.proto.scheduler.v1.JobMetadata 0, // 11: dapr.proto.scheduler.v1.GetJobResponse.job:type_name -> dapr.proto.scheduler.v1.Job 4, // 12: dapr.proto.scheduler.v1.DeleteJobRequest.metadata:type_name -> dapr.proto.scheduler.v1.JobMetadata - 0, // 13: dapr.proto.scheduler.v1.NamedJob.job:type_name -> dapr.proto.scheduler.v1.Job - 4, // 14: dapr.proto.scheduler.v1.ListJobsRequest.metadata:type_name -> dapr.proto.scheduler.v1.JobMetadata - 15, // 15: dapr.proto.scheduler.v1.ListJobsResponse.jobs:type_name -> dapr.proto.scheduler.v1.NamedJob - 9, // 16: dapr.proto.scheduler.v1.Scheduler.ScheduleJob:input_type -> dapr.proto.scheduler.v1.ScheduleJobRequest - 11, // 17: dapr.proto.scheduler.v1.Scheduler.GetJob:input_type -> dapr.proto.scheduler.v1.GetJobRequest - 13, // 18: dapr.proto.scheduler.v1.Scheduler.DeleteJob:input_type -> dapr.proto.scheduler.v1.DeleteJobRequest - 5, // 19: dapr.proto.scheduler.v1.Scheduler.WatchJobs:input_type -> dapr.proto.scheduler.v1.WatchJobsRequest - 16, // 20: dapr.proto.scheduler.v1.Scheduler.ListJobs:input_type -> dapr.proto.scheduler.v1.ListJobsRequest - 10, // 21: dapr.proto.scheduler.v1.Scheduler.ScheduleJob:output_type -> dapr.proto.scheduler.v1.ScheduleJobResponse - 12, // 22: dapr.proto.scheduler.v1.Scheduler.GetJob:output_type -> dapr.proto.scheduler.v1.GetJobResponse - 14, // 23: dapr.proto.scheduler.v1.Scheduler.DeleteJob:output_type -> dapr.proto.scheduler.v1.DeleteJobResponse - 8, // 24: dapr.proto.scheduler.v1.Scheduler.WatchJobs:output_type -> dapr.proto.scheduler.v1.WatchJobsResponse - 17, // 25: dapr.proto.scheduler.v1.Scheduler.ListJobs:output_type -> dapr.proto.scheduler.v1.ListJobsResponse - 21, // [21:26] is the sub-list for method output_type - 16, // [16:21] is the sub-list for method input_type - 16, // [16:16] is the sub-list for extension type_name - 16, // [16:16] is the sub-list for extension extendee - 0, // [0:16] is the sub-list for field type_name + 4, // 13: dapr.proto.scheduler.v1.NamedJob.metadata:type_name -> dapr.proto.scheduler.v1.JobMetadata + 0, // 14: dapr.proto.scheduler.v1.NamedJob.job:type_name -> dapr.proto.scheduler.v1.Job + 4, // 15: dapr.proto.scheduler.v1.ListJobsRequest.metadata:type_name -> dapr.proto.scheduler.v1.JobMetadata + 15, // 16: dapr.proto.scheduler.v1.ListJobsResponse.jobs:type_name -> dapr.proto.scheduler.v1.NamedJob + 9, // 17: dapr.proto.scheduler.v1.Scheduler.ScheduleJob:input_type -> dapr.proto.scheduler.v1.ScheduleJobRequest + 11, // 18: dapr.proto.scheduler.v1.Scheduler.GetJob:input_type -> dapr.proto.scheduler.v1.GetJobRequest + 13, // 19: dapr.proto.scheduler.v1.Scheduler.DeleteJob:input_type -> dapr.proto.scheduler.v1.DeleteJobRequest + 5, // 20: dapr.proto.scheduler.v1.Scheduler.WatchJobs:input_type -> dapr.proto.scheduler.v1.WatchJobsRequest + 16, // 21: dapr.proto.scheduler.v1.Scheduler.ListJobs:input_type -> dapr.proto.scheduler.v1.ListJobsRequest + 10, // 22: dapr.proto.scheduler.v1.Scheduler.ScheduleJob:output_type -> dapr.proto.scheduler.v1.ScheduleJobResponse + 12, // 23: dapr.proto.scheduler.v1.Scheduler.GetJob:output_type -> dapr.proto.scheduler.v1.GetJobResponse + 14, // 24: dapr.proto.scheduler.v1.Scheduler.DeleteJob:output_type -> dapr.proto.scheduler.v1.DeleteJobResponse + 8, // 25: dapr.proto.scheduler.v1.Scheduler.WatchJobs:output_type -> dapr.proto.scheduler.v1.WatchJobsResponse + 17, // 26: dapr.proto.scheduler.v1.Scheduler.ListJobs:output_type -> dapr.proto.scheduler.v1.ListJobsResponse + 22, // [22:27] is the sub-list for method output_type + 17, // [17:22] is the sub-list for method input_type + 17, // [17:17] is the sub-list for extension type_name + 17, // [17:17] is the sub-list for extension extendee + 0, // [0:17] is the sub-list for field type_name } func init() { file_dapr_proto_scheduler_v1_scheduler_proto_init() } diff --git a/pkg/runtime/runtime.go b/pkg/runtime/runtime.go index fc1d81217e8..f5e881c5c17 100644 --- a/pkg/runtime/runtime.go +++ b/pkg/runtime/runtime.go @@ -1088,6 +1088,7 @@ func (a *DaprRuntime) initActors(ctx context.Context) error { Security: a.sec, SchedulerClients: a.schedulerClients, SchedulerReminders: a.globalConfig.IsFeatureEnabled(config.SchedulerReminders), + Healthz: a.runtimeConfig.healthz, }) if err != nil { return rterrors.NewInit(rterrors.InitFailure, "actors", err) diff --git a/pkg/runtime/wfengine/backends/actors/backend_test.go b/pkg/runtime/wfengine/backends/actors/backend_test.go index 23ef1b28a69..311c99118bf 100644 --- a/pkg/runtime/wfengine/backends/actors/backend_test.go +++ b/pkg/runtime/wfengine/backends/actors/backend_test.go @@ -31,6 +31,7 @@ import ( "github.com/dapr/dapr/pkg/actors" "github.com/dapr/dapr/pkg/components/wfbackend" "github.com/dapr/dapr/pkg/config" + "github.com/dapr/dapr/pkg/healthz" "github.com/dapr/dapr/pkg/resiliency" "github.com/dapr/dapr/pkg/runtime/compstore" daprt "github.com/dapr/dapr/pkg/testing" @@ -240,6 +241,7 @@ func getActorRuntime(t *testing.T) actors.Actors { StateStoreName: "workflowStore", MockPlacement: actors.NewMockPlacement(testAppID), Resiliency: resiliency.New(logger.NewLogger("test")), + Healthz: healthz.New(), }) require.NoError(t, err) return act diff --git a/pkg/runtime/wfengine/wfengine_test.go b/pkg/runtime/wfengine/wfengine_test.go index 86e4f83eec3..a68b482bf06 100644 --- a/pkg/runtime/wfengine/wfengine_test.go +++ b/pkg/runtime/wfengine/wfengine_test.go @@ -40,6 +40,7 @@ import ( "github.com/dapr/components-contrib/state" "github.com/dapr/dapr/pkg/actors" "github.com/dapr/dapr/pkg/config" + "github.com/dapr/dapr/pkg/healthz" "github.com/dapr/dapr/pkg/resiliency" "github.com/dapr/dapr/pkg/runtime/compstore" "github.com/dapr/dapr/pkg/runtime/processor" @@ -944,6 +945,7 @@ func getEngine(t *testing.T, ctx context.Context) *wfengine.WorkflowEngine { StateStoreName: "workflowStore", MockPlacement: actors.NewMockPlacement(testAppID), Resiliency: resiliency.New(logger.NewLogger("test")), + Healthz: healthz.New(), }) require.NoError(t, err) @@ -979,6 +981,7 @@ func getEngineAndStateStore(t *testing.T, ctx context.Context) (*wfengine.Workfl StateStoreName: "workflowStore", MockPlacement: actors.NewMockPlacement(testAppID), Resiliency: resiliency.New(logger.NewLogger("test")), + Healthz: healthz.New(), }) require.NoError(t, err) diff --git a/pkg/scheduler/server/api.go b/pkg/scheduler/server/api.go index a200208823d..432f3d3585e 100644 --- a/pkg/scheduler/server/api.go +++ b/pkg/scheduler/server/api.go @@ -23,6 +23,7 @@ import ( schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" "github.com/dapr/dapr/pkg/scheduler/monitoring" + "github.com/dapr/dapr/pkg/scheduler/server/internal/serialize" ) func (s *Server) ScheduleJob(ctx context.Context, req *schedulerv1pb.ScheduleJobRequest) (*schedulerv1pb.ScheduleJobResponse, error) { @@ -127,8 +128,14 @@ func (s *Server) ListJobs(ctx context.Context, req *schedulerv1pb.ListJobsReques jobs := make([]*schedulerv1pb.NamedJob, 0, len(list.GetJobs())) for _, job := range list.GetJobs() { + meta, err := serialize.MetadataFromKey(job.GetName()) + if err != nil { + return nil, fmt.Errorf("failed to parse job metadata: %w", err) + } + jobs = append(jobs, &schedulerv1pb.NamedJob{ - Name: job.GetName()[strings.LastIndex(job.GetName(), "||")+2:], + Name: job.GetName()[strings.LastIndex(job.GetName(), "||")+2:], + Metadata: meta, //nolint:protogetter Job: &schedulerv1pb.Job{ Schedule: job.GetJob().Schedule, diff --git a/pkg/scheduler/server/internal/serialize/names.go b/pkg/scheduler/server/internal/serialize/names.go index 8bece16e960..6b7fb8be205 100644 --- a/pkg/scheduler/server/internal/serialize/names.go +++ b/pkg/scheduler/server/internal/serialize/names.go @@ -13,9 +13,52 @@ limitations under the License. package serialize +import ( + "fmt" + "path" + "strings" + + schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" +) + +// PrefixesFromNamespace returns key prefixes for all jobs types for a given +// namespace. func PrefixesFromNamespace(namespace string) []string { return []string{ "actorreminder||" + namespace, "app||" + namespace, } } + +// MetadataFromKey returns the JobMetadata based on a raw job key. +func MetadataFromKey(key string) (*schedulerv1pb.JobMetadata, error) { + seg := strings.Split(key, "||") + + if len(seg) >= 5 && path.Base(seg[len(seg)-5]) == "actorreminder" { + seg = seg[len(seg)-4:] + return &schedulerv1pb.JobMetadata{ + Namespace: seg[0], + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Actor{ + Actor: &schedulerv1pb.TargetActorReminder{ + Type: seg[1], + Id: seg[2], + }, + }, + }, + }, nil + } + + if len(seg) >= 4 && path.Base(seg[len(seg)-4]) == "app" { + seg = seg[len(seg)-3:] + return &schedulerv1pb.JobMetadata{ + Namespace: seg[0], + AppId: seg[1], + Target: &schedulerv1pb.JobTargetMetadata{ + Type: new(schedulerv1pb.JobTargetMetadata_Job), + }, + }, nil + } + + return nil, fmt.Errorf("invalid key: %s", key) +} diff --git a/pkg/scheduler/server/internal/serialize/names_test.go b/pkg/scheduler/server/internal/serialize/names_test.go new file mode 100644 index 00000000000..6f373c8b877 --- /dev/null +++ b/pkg/scheduler/server/internal/serialize/names_test.go @@ -0,0 +1,138 @@ +/* +Copyright 2024 The Dapr 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 serialize + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "google.golang.org/protobuf/proto" + + schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" +) + +func Test_MetadataFromKey(t *testing.T) { + t.Parallel() + + tests := map[string]struct { + key string + expMeta *schedulerv1pb.JobMetadata + expErr bool + }{ + "an empty key should error": { + key: "", + expMeta: nil, + expErr: true, + }, + "a random key should error": { + key: "random", + expMeta: nil, + expErr: true, + }, + "a bad reminder should error": { + key: "foo||bar||actorreminde||myns||myactortype||myactorid||myremindername", + expMeta: nil, + expErr: true, + }, + "a no-namespace actor reminder should return metadata": { + key: "actorreminder||myns||myactortype||myactorid||myremindername", + expMeta: &schedulerv1pb.JobMetadata{ + AppId: "", Namespace: "myns", + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Actor{ + Actor: &schedulerv1pb.TargetActorReminder{ + Id: "myactorid", Type: "myactortype", + }, + }, + }, + }, + }, + "a namespace actor reminder should return metadata": { + key: "foo/bar/actorreminder||myns||myactortype||myactorid||myremindername", + expMeta: &schedulerv1pb.JobMetadata{ + AppId: "", Namespace: "myns", + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Actor{ + Actor: &schedulerv1pb.TargetActorReminder{ + Id: "myactorid", Type: "myactortype", + }, + }, + }, + }, + }, + "a namespace (with separator) actor reminder should return metadata": { + key: "foo||bar/actorreminder||myns||myactortype||myactorid||myremindername", + expMeta: &schedulerv1pb.JobMetadata{ + AppId: "", Namespace: "myns", + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Actor{ + Actor: &schedulerv1pb.TargetActorReminder{ + Id: "myactorid", Type: "myactortype", + }, + }, + }, + }, + }, + "a namespace (with separator 2) actor reminder should return metadata": { + key: "foo||bar||actorreminder||myns||myactortype||myactorid||myremindername", + expMeta: &schedulerv1pb.JobMetadata{ + AppId: "", Namespace: "myns", + Target: &schedulerv1pb.JobTargetMetadata{ + Type: &schedulerv1pb.JobTargetMetadata_Actor{ + Actor: &schedulerv1pb.TargetActorReminder{ + Id: "myactorid", Type: "myactortype", + }, + }, + }, + }, + }, + "a no-namespace job should return metadata": { + key: "app||myns||myid||jobname", + expMeta: &schedulerv1pb.JobMetadata{ + AppId: "myid", Namespace: "myns", + Target: &schedulerv1pb.JobTargetMetadata{Type: new(schedulerv1pb.JobTargetMetadata_Job)}, + }, + }, + "a namespace job should return metadata": { + key: "foo/bar/app||myns||myid||jobname", + expMeta: &schedulerv1pb.JobMetadata{ + AppId: "myid", Namespace: "myns", + Target: &schedulerv1pb.JobTargetMetadata{Type: new(schedulerv1pb.JobTargetMetadata_Job)}, + }, + }, + "a namespace (with separator) job should return metadata": { + key: "foo||bar/app||myns||myid||jobname", + expMeta: &schedulerv1pb.JobMetadata{ + AppId: "myid", Namespace: "myns", + Target: &schedulerv1pb.JobTargetMetadata{Type: new(schedulerv1pb.JobTargetMetadata_Job)}, + }, + }, + "a namespace (with separator 2) job should return metadata": { + key: "foo||bar||app||myns||myid||jobname", + expMeta: &schedulerv1pb.JobMetadata{ + AppId: "myid", Namespace: "myns", + Target: &schedulerv1pb.JobTargetMetadata{Type: new(schedulerv1pb.JobTargetMetadata_Job)}, + }, + }, + } + + for name, test := range tests { + t.Run(name, func(t *testing.T) { + t.Parallel() + got, err := MetadataFromKey(test.key) + assert.Equal(t, test.expErr, err != nil) + assert.True(t, proto.Equal(test.expMeta, got), "expected: %v, got: %v", test.expMeta, got) + }) + } +} diff --git a/tests/integration/framework/process/daprd/actors/actors.go b/tests/integration/framework/process/daprd/actors/actors.go new file mode 100644 index 00000000000..6e0720ac06e --- /dev/null +++ b/tests/integration/framework/process/daprd/actors/actors.go @@ -0,0 +1,135 @@ +/* +Copyright 2024 The Dapr 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 actors + +import ( + "context" + "fmt" + "runtime" + "strings" + "testing" + + "google.golang.org/grpc" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework/process/daprd" + "github.com/dapr/dapr/tests/integration/framework/process/http/app" + "github.com/dapr/dapr/tests/integration/framework/process/placement" + "github.com/dapr/dapr/tests/integration/framework/process/scheduler" + "github.com/dapr/dapr/tests/integration/framework/process/sqlite" +) + +type Actors struct { + app *app.App + db *sqlite.SQLite + place *placement.Placement + sched *scheduler.Scheduler + daprd *daprd.Daprd +} + +func New(t *testing.T, fopts ...Option) *Actors { + t.Helper() + + if runtime.GOOS == "windows" { + t.Skip("Skipping test on Windows due to SQLite limitations") + } + + opts := options{ + db: sqlite.New(t, + sqlite.WithActorStateStore(true), + sqlite.WithCreateStateTables(), + ), + placement: placement.New(t), + scheduler: scheduler.New(t), + } + for _, fopt := range fopts { + fopt(&opts) + } + + handlers := make([]app.Option, 0, len(opts.actorTypeHandlers)) + for atype, handler := range opts.actorTypeHandlers { + handlers = append(handlers, app.WithHandlerFunc("/actors/"+atype+"/", handler)) + } + + app := app.New(t, + append(handlers, + app.WithConfig(fmt.Sprintf(`{"entities": [%s]}`, strings.Join(opts.types, ","))), + )..., + ) + + dopts := []daprd.Option{ + daprd.WithAppPort(app.Port()), + daprd.WithPlacementAddresses(opts.placement.Address()), + daprd.WithResourceFiles(opts.db.GetComponent(t)), + daprd.WithConfigManifests(t, opts.daprdConfigs...), + daprd.WithScheduler(opts.scheduler), + } + + return &Actors{ + app: app, + db: opts.db, + place: opts.placement, + sched: opts.scheduler, + daprd: daprd.New(t, dopts...), + } +} + +func (a *Actors) Run(t *testing.T, ctx context.Context) { + a.app.Run(t, ctx) + a.db.Run(t, ctx) + a.place.Run(t, ctx) + a.sched.Run(t, ctx) + a.daprd.Run(t, ctx) +} + +func (a *Actors) Cleanup(t *testing.T) { + a.daprd.Cleanup(t) + a.sched.Cleanup(t) + a.place.Cleanup(t) + a.db.Cleanup(t) + a.app.Cleanup(t) +} + +func (a *Actors) WaitUntilRunning(t *testing.T, ctx context.Context) { + a.place.WaitUntilRunning(t, ctx) + a.sched.WaitUntilRunning(t, ctx) + a.daprd.WaitUntilRunning(t, ctx) +} + +func (a *Actors) GRPCClient(t *testing.T, ctx context.Context) rtv1.DaprClient { + t.Helper() + return a.daprd.GRPCClient(t, ctx) +} + +func (a *Actors) GRPCConn(t *testing.T, ctx context.Context) *grpc.ClientConn { + t.Helper() + return a.daprd.GRPCConn(t, ctx) +} + +func (a *Actors) Metrics(t *testing.T, ctx context.Context) map[string]float64 { + t.Helper() + return a.daprd.Metrics(t, ctx) +} + +func (a *Actors) Placement() *placement.Placement { + return a.place +} + +func (a *Actors) Scheduler() *scheduler.Scheduler { + return a.sched +} + +func (a *Actors) DB() *sqlite.SQLite { + return a.db +} diff --git a/tests/integration/framework/process/daprd/actors/options.go b/tests/integration/framework/process/daprd/actors/options.go new file mode 100644 index 00000000000..fabb432da4d --- /dev/null +++ b/tests/integration/framework/process/daprd/actors/options.go @@ -0,0 +1,84 @@ +/* +Copyright 2024 The Dapr 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 actors + +import ( + "net/http" + "strconv" + + "github.com/dapr/dapr/tests/integration/framework/process/placement" + "github.com/dapr/dapr/tests/integration/framework/process/scheduler" + "github.com/dapr/dapr/tests/integration/framework/process/sqlite" +) + +type Option func(*options) + +type options struct { + db *sqlite.SQLite + types []string + + placement *placement.Placement + scheduler *scheduler.Scheduler + daprdConfigs []string + actorTypeHandlers map[string]http.HandlerFunc +} + +func WithDB(db *sqlite.SQLite) Option { + return func(o *options) { + o.db = db + } +} + +func WithActorTypes(types ...string) Option { + return func(o *options) { + for _, atype := range types { + o.types = append(o.types, `"`+atype+`"`) + } + } +} + +func WithPlacement(placement *placement.Placement) Option { + return func(o *options) { + o.placement = placement + } +} + +func WithScheduler(scheduler *scheduler.Scheduler) Option { + return func(o *options) { + o.scheduler = scheduler + } +} + +func WithFeatureSchedulerReminders(enabled bool) Option { + return func(o *options) { + o.daprdConfigs = append(o.daprdConfigs, ` +apiVersion: dapr.io/v1alpha1 +kind: Configuration +metadata: + name: appconfig +spec: + features: + - name: SchedulerReminders + enabled: `+strconv.FormatBool(enabled)) + } +} + +func WithActorTypeHandler(actorType string, handler http.HandlerFunc) Option { + return func(o *options) { + if o.actorTypeHandlers == nil { + o.actorTypeHandlers = make(map[string]http.HandlerFunc) + } + o.actorTypeHandlers[actorType] = handler + } +} diff --git a/tests/integration/framework/process/daprd/workflow/options.go b/tests/integration/framework/process/daprd/workflow/options.go new file mode 100644 index 00000000000..cd251d66354 --- /dev/null +++ b/tests/integration/framework/process/daprd/workflow/options.go @@ -0,0 +1,51 @@ +/* +Copyright 2024 The Dapr 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 workflow + +import ( + "testing" + + "github.com/microsoft/durabletask-go/task" + "github.com/stretchr/testify/require" +) + +type Option func(*options) + +type options struct { + registry *task.TaskRegistry + + enableScheduler bool +} + +func WithScheduler(enable bool) Option { + return func(o *options) { + o.enableScheduler = enable + } +} + +func WithAddOrchestratorN(t *testing.T, name string, or func(*task.OrchestrationContext) (any, error)) Option { + t.Helper() + + return func(o *options) { + require.NoError(t, o.registry.AddOrchestratorN(name, or)) + } +} + +func WithAddActivityN(t *testing.T, name string, a func(task.ActivityContext) (any, error)) Option { + t.Helper() + + return func(o *options) { + require.NoError(t, o.registry.AddActivityN(name, a)) + } +} diff --git a/tests/integration/framework/process/daprd/workflow/workflow.go b/tests/integration/framework/process/daprd/workflow/workflow.go new file mode 100644 index 00000000000..bd957972c43 --- /dev/null +++ b/tests/integration/framework/process/daprd/workflow/workflow.go @@ -0,0 +1,77 @@ +/* +Copyright 2024 The Dapr 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 workflow + +import ( + "context" + "testing" + + "github.com/microsoft/durabletask-go/client" + "github.com/microsoft/durabletask-go/task" + "github.com/stretchr/testify/require" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework/iowriter/logger" + "github.com/dapr/dapr/tests/integration/framework/process/daprd/actors" +) + +type Workflow struct { + registry *task.TaskRegistry + actors *actors.Actors +} + +func New(t *testing.T, fopts ...Option) *Workflow { + t.Helper() + + opts := options{ + registry: task.NewTaskRegistry(), + } + for _, fopt := range fopts { + fopt(&opts) + } + + return &Workflow{ + registry: opts.registry, + actors: actors.New(t, actors.WithFeatureSchedulerReminders(opts.enableScheduler)), + } +} + +func (w *Workflow) Run(t *testing.T, ctx context.Context) { + w.actors.Run(t, ctx) +} + +func (w *Workflow) Cleanup(t *testing.T) { + w.actors.Cleanup(t) +} + +func (w *Workflow) WaitUntilRunning(t *testing.T, ctx context.Context) { + w.actors.WaitUntilRunning(t, ctx) +} + +func (w *Workflow) BackendClient(t *testing.T, ctx context.Context) *client.TaskHubGrpcClient { + t.Helper() + backendClient := client.NewTaskHubGrpcClient(w.actors.GRPCConn(t, ctx), logger.New(t)) + require.NoError(t, backendClient.StartWorkItemListener(ctx, w.registry)) + return backendClient +} + +func (w *Workflow) GRPCClient(t *testing.T, ctx context.Context) rtv1.DaprClient { + t.Helper() + return w.actors.GRPCClient(t, ctx) +} + +func (w *Workflow) Metrics(t *testing.T, ctx context.Context) map[string]float64 { + t.Helper() + return w.actors.Metrics(t, ctx) +} diff --git a/tests/integration/framework/process/placement/placement.go b/tests/integration/framework/process/placement/placement.go index 101227b596e..16bf2a6be8e 100644 --- a/tests/integration/framework/process/placement/placement.go +++ b/tests/integration/framework/process/placement/placement.go @@ -19,7 +19,7 @@ import ( "fmt" "net/http" "strconv" - "sync/atomic" + "sync" "testing" "time" @@ -42,9 +42,8 @@ import ( ) type Placement struct { - exec process.Interface - ports *ports.Ports - running atomic.Bool + exec process.Interface + ports *ports.Ports id string port int @@ -52,6 +51,9 @@ type Placement struct { metricsPort int initialCluster string initialClusterPorts []int + + runOnce sync.Once + cleanupOnce sync.Once } func New(t *testing.T, fopts ...Option) *Placement { @@ -116,20 +118,16 @@ func New(t *testing.T, fopts ...Option) *Placement { } func (p *Placement) Run(t *testing.T, ctx context.Context) { - if !p.running.CompareAndSwap(false, true) { - t.Fatal("Process is already running") - } - - p.ports.Free(t) - p.exec.Run(t, ctx) + p.runOnce.Do(func() { + p.ports.Free(t) + p.exec.Run(t, ctx) + }) } func (p *Placement) Cleanup(t *testing.T) { - if !p.running.CompareAndSwap(true, false) { - return - } - - p.exec.Cleanup(t) + p.cleanupOnce.Do(func() { + p.exec.Cleanup(t) + }) } func (p *Placement) WaitUntilRunning(t *testing.T, ctx context.Context) { diff --git a/tests/integration/framework/process/scheduler/scheduler.go b/tests/integration/framework/process/scheduler/scheduler.go index 2d0475d4bc7..32c54fe52be 100644 --- a/tests/integration/framework/process/scheduler/scheduler.go +++ b/tests/integration/framework/process/scheduler/scheduler.go @@ -22,7 +22,7 @@ import ( "path/filepath" "strconv" "strings" - "sync/atomic" + "sync" "testing" "time" @@ -52,7 +52,6 @@ import ( type Scheduler struct { exec process.Interface ports *ports.Ports - running atomic.Bool httpClient *http.Client port int @@ -65,6 +64,9 @@ type Scheduler struct { initialCluster string etcdClientPorts map[string]string sentry *sentry.Sentry + + runOnce sync.Once + cleanupOnce sync.Once } func New(t *testing.T, fopts ...Option) *Scheduler { @@ -165,20 +167,16 @@ func New(t *testing.T, fopts ...Option) *Scheduler { } func (s *Scheduler) Run(t *testing.T, ctx context.Context) { - if !s.running.CompareAndSwap(false, true) { - t.Fatal("Process is already running") - } - - s.ports.Free(t) - s.exec.Run(t, ctx) + s.runOnce.Do(func() { + s.ports.Free(t) + s.exec.Run(t, ctx) + }) } func (s *Scheduler) Cleanup(t *testing.T) { - if !s.running.CompareAndSwap(true, false) { - return - } - - s.exec.Cleanup(t) + s.cleanupOnce.Do(func() { + s.exec.Cleanup(t) + }) } func (s *Scheduler) WaitUntilRunning(t *testing.T, ctx context.Context) { diff --git a/tests/integration/framework/process/sqlite/actors.go b/tests/integration/framework/process/sqlite/actors.go new file mode 100644 index 00000000000..d286f037d9a --- /dev/null +++ b/tests/integration/framework/process/sqlite/actors.go @@ -0,0 +1,83 @@ +/* +Copyright 2024 The Dapr 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 sqlite + +import ( + "context" + "encoding/json" + "fmt" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +type ActorTypeReminders struct { + Key string + Reminders []ActorReminder + IsBinary bool + Etag string + ExpirationTime *time.Time + UpdateTime *time.Time +} + +type ActorReminder struct { + RegisteredTime time.Time + ExpirationTime time.Time + Period string + Data string + ActorID string + ActorType string + Name string + DueTime string +} + +type internalActorTypeReminders struct { + Key string + Value string + IsBinary bool + Etag string + ExpirationTime *time.Time + UpdateTime *time.Time +} + +func (s *SQLite) ActorReminders(t *testing.T, ctx context.Context, actorType string) ActorTypeReminders { + t.Helper() + + query := fmt.Sprintf("SELECT * FROM '%s' WHERE key = 'actors||%s'", s.tableName, actorType) + rows, err := s.GetConnection(t).QueryContext(ctx, query) + require.NoError(t, err) + + require.NoError(t, rows.Err()) + + if !rows.Next() { + return ActorTypeReminders{} + } + + var r internalActorTypeReminders + require.NoError(t, rows.Scan(&r.Key, &r.Value, &r.IsBinary, &r.Etag, &r.ExpirationTime, &r.UpdateTime)) + require.False(t, rows.Next(), "Reminder should only have a single entry per actor type") + + var reminders []ActorReminder + require.NoError(t, json.Unmarshal([]byte(r.Value), &reminders)) + + return ActorTypeReminders{ + Key: r.Key, + Reminders: reminders, + IsBinary: r.IsBinary, + Etag: r.Etag, + ExpirationTime: r.ExpirationTime, + UpdateTime: r.UpdateTime, + } +} diff --git a/tests/integration/framework/process/sqlite/options.go b/tests/integration/framework/process/sqlite/options.go index a69efd9804a..c1f9273e225 100644 --- a/tests/integration/framework/process/sqlite/options.go +++ b/tests/integration/framework/process/sqlite/options.go @@ -13,13 +13,17 @@ limitations under the License. package sqlite +import "fmt" + // options contains the options for using a SQLite database in integration tests. type options struct { name string isActorStateStore bool metadata map[string]string - migrations []string + migrations []func(tableName string) string execs []string + dbPath string + tableName string } // WithName sets the name for the state store. @@ -51,20 +55,22 @@ func WithMetadata(key, value string) Option { // WithCreateStateTables configures whether the state store should create the state tables. func WithCreateStateTables() Option { return func(o *options) { - o.migrations = append(o.migrations, ` + o.migrations = append(o.migrations, func(tableName string) string { + return fmt.Sprintf(` CREATE TABLE metadata ( key text NOT NULL PRIMARY KEY, value text NOT NULL ); INSERT INTO metadata VALUES('migrations','1'); -CREATE TABLE state ( +CREATE TABLE %s ( key TEXT NOT NULL PRIMARY KEY, value TEXT NOT NULL, is_binary BOOLEAN NOT NULL, etag TEXT NOT NULL, expiration_time TIMESTAMP DEFAULT NULL, update_time TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP -);`) +);`, tableName) + }) } } @@ -74,3 +80,15 @@ func WithExecs(execs ...string) Option { o.execs = execs } } + +func WithDBPath(dbPath string) Option { + return func(o *options) { + o.dbPath = dbPath + } +} + +func WithTableName(tableName string) Option { + return func(o *options) { + o.tableName = tableName + } +} diff --git a/tests/integration/framework/process/sqlite/sqlite.go b/tests/integration/framework/process/sqlite/sqlite.go index 92a5ed81dcf..49c293e528c 100644 --- a/tests/integration/framework/process/sqlite/sqlite.go +++ b/tests/integration/framework/process/sqlite/sqlite.go @@ -19,6 +19,7 @@ import ( "encoding/json" "path/filepath" "strconv" + "sync" "testing" // Blank import for the sqlite driver @@ -42,57 +43,68 @@ type SQLite struct { dbPath string name string metadata map[string]string - migrations []string + migrations []func(string) string isActorStateStore bool execs []string conn *sql.DB + tableName string + lock sync.Mutex + runOnce sync.Once + cleanupOnce sync.Once } func New(t *testing.T, fopts ...Option) *SQLite { t.Helper() opts := options{ - name: "mystore", + name: "mystore", + dbPath: filepath.Join(t.TempDir(), "test-data.db"), + tableName: "inttest", } - for _, fopt := range fopts { fopt(&opts) } // Create a SQLite database in the test's temporary directory - dbPath := filepath.Join(t.TempDir(), "test-data.db") - t.Logf("Storing SQLite database at %s", dbPath) + t.Logf("Storing SQLite database at %s", opts.dbPath) return &SQLite{ - dbPath: dbPath, + dbPath: opts.dbPath, name: opts.name, metadata: opts.metadata, migrations: opts.migrations, isActorStateStore: opts.isActorStateStore, execs: opts.execs, + tableName: opts.tableName, } } func (s *SQLite) Run(t *testing.T, ctx context.Context) { - for _, migration := range s.migrations { - _, err := s.GetConnection(t).ExecContext(ctx, migration) - require.NoError(t, err) - } - - for _, exec := range s.execs { - _, err := s.GetConnection(t).ExecContext(ctx, exec) - require.NoError(t, err) - } + s.runOnce.Do(func() { + for _, migration := range s.migrations { + _, err := s.GetConnection(t).ExecContext(ctx, migration(s.tableName)) + require.NoError(t, err) + } + + for _, exec := range s.execs { + _, err := s.GetConnection(t).ExecContext(ctx, exec) + require.NoError(t, err) + } + }) } func (s *SQLite) Cleanup(t *testing.T) { - if s.conn != nil { - require.NoError(t, s.conn.Close()) - } + s.cleanupOnce.Do(func() { + if s.conn != nil { + require.NoError(t, s.conn.Close()) + } + }) } // GetConnection returns the connection to the SQLite database. func (s *SQLite) GetConnection(t *testing.T) *sql.DB { + s.lock.Lock() + defer s.lock.Unlock() if s.conn != nil { return s.conn } @@ -118,6 +130,7 @@ func (s *SQLite) GetComponent(t *testing.T) string { Metadata: []commonapi.NameValuePair{ {Name: "connectionString", Value: toDynamicValue(t, "file:"+s.dbPath)}, {Name: "actorStateStore", Value: toDynamicValue(t, strconv.FormatBool(s.isActorStateStore))}, + {Name: "tableName", Value: toDynamicValue(t, s.tableName)}, }, }, } @@ -134,6 +147,10 @@ func (s *SQLite) GetComponent(t *testing.T) string { return string(enc) } +func (s *SQLite) TableName() string { + return s.tableName +} + func toDynamicValue(t *testing.T, val string) commonapi.DynamicValue { j, err := json.Marshal(val) require.NoError(t, err) diff --git a/tests/integration/suite/actors/reminders/data.go b/tests/integration/suite/actors/reminders/data.go new file mode 100644 index 00000000000..ff9da36b7c6 --- /dev/null +++ b/tests/integration/suite/actors/reminders/data.go @@ -0,0 +1,77 @@ +/* +Copyright 2024 The Dapr 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 impliei. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package reminders + +import ( + "context" + "io" + "net/http" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/daprd/actors" + "github.com/dapr/dapr/tests/integration/suite" +) + +func init() { + suite.Register(new(data)) +} + +type data struct { + actors *actors.Actors + got chan string +} + +func (d *data) Setup(t *testing.T) []framework.Option { + d.got = make(chan string, 1) + d.actors = actors.New(t, + actors.WithActorTypes("foo"), + actors.WithActorTypeHandler("foo", func(_ http.ResponseWriter, req *http.Request) { + got, err := io.ReadAll(req.Body) + assert.NoError(t, err) + d.got <- string(got) + }), + ) + + return []framework.Option{ + framework.WithProcesses(d.actors), + } +} + +func (d *data) Run(t *testing.T, ctx context.Context) { + d.actors.WaitUntilRunning(t, ctx) + + _, err := d.actors.GRPCClient(t, ctx).RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "foo", + ActorId: "1234", + Name: "helloworld", + DueTime: "0s", + Period: "1000s", + Ttl: "2000s", + Data: []byte("mydata"), + }) + require.NoError(t, err) + + select { + case got := <-d.got: + assert.JSONEq(t, `{"data":"bXlkYXRh","dueTime":"0s","period":"1000s"}`, got) + case <-time.After(10 * time.Second): + t.Fatal("timed out waiting for reminder") + } +} diff --git a/tests/integration/suite/actors/reminders/migration/migration.go b/tests/integration/suite/actors/reminders/migration/migration.go new file mode 100644 index 00000000000..d1497ea6b63 --- /dev/null +++ b/tests/integration/suite/actors/reminders/migration/migration.go @@ -0,0 +1,18 @@ +/* +Copyright 2024 The Dapr 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 migration + +import ( + _ "github.com/dapr/dapr/tests/integration/suite/actors/reminders/migration/scheduler" +) diff --git a/tests/integration/suite/actors/reminders/migration/scheduler/basic.go b/tests/integration/suite/actors/reminders/migration/scheduler/basic.go new file mode 100644 index 00000000000..681e28bf9c5 --- /dev/null +++ b/tests/integration/suite/actors/reminders/migration/scheduler/basic.go @@ -0,0 +1,106 @@ +/* +Copyright 2024 The Dapr 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 scheduler + +import ( + "context" + "net/http" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/daprd" + "github.com/dapr/dapr/tests/integration/framework/process/http/app" + "github.com/dapr/dapr/tests/integration/framework/process/placement" + "github.com/dapr/dapr/tests/integration/framework/process/scheduler" + "github.com/dapr/dapr/tests/integration/framework/process/sqlite" + "github.com/dapr/dapr/tests/integration/suite" +) + +func init() { + suite.Register(new(basic)) +} + +type basic struct { + db *sqlite.SQLite + app *app.App + place *placement.Placement + scheduler *scheduler.Scheduler +} + +func (b *basic) Setup(t *testing.T) []framework.Option { + b.db = sqlite.New(t, sqlite.WithActorStateStore(true)) + b.app = app.New(t, + app.WithConfig(`{"entities": ["myactortype"]}`), + app.WithHandlerFunc("/actors/myactortype/myactorid", func(http.ResponseWriter, *http.Request) {}), + ) + b.scheduler = scheduler.New(t) + b.place = placement.New(t) + + return []framework.Option{ + framework.WithProcesses(b.db, b.scheduler, b.place, b.app), + } +} + +func (b *basic) Run(t *testing.T, ctx context.Context) { + opts := []daprd.Option{ + daprd.WithResourceFiles(b.db.GetComponent(t)), + daprd.WithPlacementAddresses(b.place.Address()), + daprd.WithSchedulerAddresses(b.scheduler.Address()), + daprd.WithAppPort(b.app.Port()), + } + + daprd1 := daprd.New(t, opts...) + daprd2 := daprd.New(t, append(opts, + daprd.WithConfigManifests(t, ` +apiVersion: dapr.io/v1alpha1 +kind: Configuration +metadata: + name: schedulerreminders +spec: + features: + - name: SchedulerReminders + enabled: true +`))...) + + daprd1.Run(t, ctx) + daprd1.WaitUntilRunning(t, ctx) + + assert.Empty(t, b.scheduler.EtcdJobs(t, ctx)) + assert.Empty(t, b.db.ActorReminders(t, ctx, "myactortype").Reminders) + + client := daprd1.GRPCClient(t, ctx) + _, err := client.RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "myactortype", + ActorId: "myactorid", + Name: "myreminder", + DueTime: "10000s", + Period: "10000s", + Data: []byte("mydata"), + Ttl: "10000s", + }) + require.NoError(t, err) + assert.Len(t, b.db.ActorReminders(t, ctx, "myactortype").Reminders, 1) + assert.Empty(t, b.scheduler.EtcdJobs(t, ctx)) + daprd1.Cleanup(t) + + daprd2.Run(t, ctx) + daprd2.WaitUntilRunning(t, ctx) + assert.Len(t, b.db.ActorReminders(t, ctx, "myactortype").Reminders, 1) + assert.Len(t, b.scheduler.EtcdJobs(t, ctx), 1) + daprd2.Cleanup(t) +} diff --git a/tests/integration/suite/actors/reminders/migration/scheduler/data.go b/tests/integration/suite/actors/reminders/migration/scheduler/data.go new file mode 100644 index 00000000000..426a19c17cd --- /dev/null +++ b/tests/integration/suite/actors/reminders/migration/scheduler/data.go @@ -0,0 +1,122 @@ +/* +Copyright 2024 The Dapr 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 scheduler + +import ( + "context" + "io" + "net/http" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/daprd" + "github.com/dapr/dapr/tests/integration/framework/process/http/app" + "github.com/dapr/dapr/tests/integration/framework/process/placement" + "github.com/dapr/dapr/tests/integration/framework/process/scheduler" + "github.com/dapr/dapr/tests/integration/framework/process/sqlite" + "github.com/dapr/dapr/tests/integration/suite" +) + +func init() { + suite.Register(new(data)) +} + +type data struct { + db *sqlite.SQLite + app *app.App + place *placement.Placement + scheduler *scheduler.Scheduler + + called atomic.Bool + got atomic.Value +} + +func (d *data) Setup(t *testing.T) []framework.Option { + d.db = sqlite.New(t, sqlite.WithActorStateStore(true)) + d.app = app.New(t, + app.WithConfig(`{"entities": ["myactortype"]}`), + app.WithHandlerFunc("/actors/myactortype/myactorid/method/remind/myreminder", func(_ http.ResponseWriter, r *http.Request) { + b, err := io.ReadAll(r.Body) + assert.NoError(t, err) + d.got.Store(b) + assert.True(t, d.called.CompareAndSwap(false, true)) + }), + ) + d.scheduler = scheduler.New(t) + d.place = placement.New(t) + + return []framework.Option{ + framework.WithProcesses(d.db, d.scheduler, d.place, d.app), + } +} + +func (d *data) Run(t *testing.T, ctx context.Context) { + opts := []daprd.Option{ + daprd.WithResourceFiles(d.db.GetComponent(t)), + daprd.WithPlacementAddresses(d.place.Address()), + daprd.WithSchedulerAddresses(d.scheduler.Address()), + daprd.WithAppPort(d.app.Port()), + } + + daprd1 := daprd.New(t, opts...) + daprd2 := daprd.New(t, append(opts, + daprd.WithConfigManifests(t, ` +apiVersion: dapr.io/v1alpha1 +kind: Configuration +metadata: + name: schedulerreminders +spec: + features: + - name: SchedulerReminders + enabled: true +`))...) + + t.Cleanup(func() { daprd1.Cleanup(t) }) + daprd1.Run(t, ctx) + daprd1.WaitUntilRunning(t, ctx) + + _, err := daprd1.GRPCClient(t, ctx).RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "myactortype", + ActorId: "myactorid", + Name: "myreminder", + DueTime: "3s", + Data: []byte("mydata"), + }) + require.NoError(t, err) + assert.Len(t, d.db.ActorReminders(t, ctx, "myactortype").Reminders, 1) + assert.Empty(t, d.scheduler.EtcdJobs(t, ctx)) + daprd1.Cleanup(t) + + assert.False(t, d.called.Load()) + + daprd2.Run(t, ctx) + daprd2.WaitUntilRunning(t, ctx) + assert.Len(t, d.db.ActorReminders(t, ctx, "myactortype").Reminders, 1) + + assert.EventuallyWithT(t, func(c *assert.CollectT) { + assert.True(c, d.called.Load()) + }, time.Second*10, time.Millisecond*10) + + got, ok := d.got.Load().([]byte) + assert.True(t, ok) + assert.JSONEq(t, `{"data":"bXlkYXRh","dueTime":"","period":""}`, string(got)) + + daprd2.Cleanup(t) +} diff --git a/tests/integration/suite/actors/reminders/migration/scheduler/distributed.go b/tests/integration/suite/actors/reminders/migration/scheduler/distributed.go new file mode 100644 index 00000000000..7a3d1be6c2e --- /dev/null +++ b/tests/integration/suite/actors/reminders/migration/scheduler/distributed.go @@ -0,0 +1,164 @@ +/* +Copyright 2024 The Dapr 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 scheduler + +import ( + "context" + "net/http" + "strconv" + "sync" + "testing" + + "github.com/stretchr/testify/assert" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/daprd" + "github.com/dapr/dapr/tests/integration/framework/process/http/app" + "github.com/dapr/dapr/tests/integration/framework/process/placement" + "github.com/dapr/dapr/tests/integration/framework/process/scheduler" + "github.com/dapr/dapr/tests/integration/framework/process/sqlite" + "github.com/dapr/dapr/tests/integration/suite" +) + +func init() { + suite.Register(new(distributed)) +} + +type distributed struct { + db *sqlite.SQLite + app1 *app.App + app2 *app.App + place *placement.Placement + scheduler *scheduler.Scheduler +} + +func (d *distributed) Setup(t *testing.T) []framework.Option { + d.db = sqlite.New(t, sqlite.WithActorStateStore(true)) + d.app1 = app.New(t, + app.WithConfig(`{"entities": ["myactortype","myactortype2"]}`), + app.WithHandlerFunc("/actors", func(http.ResponseWriter, *http.Request) {}), + ) + d.app2 = app.New(t, + app.WithConfig(`{"entities": ["myactortype"]}`), + app.WithHandlerFunc("/actors", func(http.ResponseWriter, *http.Request) {}), + ) + d.scheduler = scheduler.New(t) + d.place = placement.New(t) + + return []framework.Option{ + framework.WithProcesses(d.db, d.scheduler, d.place, d.app1, d.app2), + } +} + +func (d *distributed) Run(t *testing.T, ctx context.Context) { + schedConfig := `apiVersion: dapr.io/v1alpha1 +kind: Configuration +metadata: + name: schedulerreminders +spec: + features: + - name: SchedulerReminders + enabled: true +` + + optsApp1 := []daprd.Option{ + daprd.WithResourceFiles(d.db.GetComponent(t)), + daprd.WithPlacementAddresses(d.place.Address()), + daprd.WithSchedulerAddresses(d.scheduler.Address()), + daprd.WithAppPort(d.app1.Port()), + } + optsApp2 := []daprd.Option{ + daprd.WithResourceFiles(d.db.GetComponent(t)), + daprd.WithPlacementAddresses(d.place.Address()), + daprd.WithSchedulerAddresses(d.scheduler.Address()), + daprd.WithAppPort(d.app2.Port()), + daprd.WithConfigManifests(t, schedConfig), + } + optsApp1WithScheduler := []daprd.Option{ + daprd.WithResourceFiles(d.db.GetComponent(t)), + daprd.WithPlacementAddresses(d.place.Address()), + daprd.WithSchedulerAddresses(d.scheduler.Address()), + daprd.WithAppPort(d.app1.Port()), + daprd.WithConfigManifests(t, schedConfig), + } + + daprd1 := daprd.New(t, optsApp1...) + daprd2 := daprd.New(t, optsApp2...) + daprd3 := daprd.New(t, optsApp2...) + daprd4 := daprd.New(t, optsApp2...) + daprd5 := daprd.New(t, optsApp2...) + daprd6 := daprd.New(t, optsApp1WithScheduler...) + + daprd1.Run(t, ctx) + daprd1.WaitUntilRunning(t, ctx) + client := daprd1.GRPCClient(t, ctx) + + var wg sync.WaitGroup + wg.Add(200) + for i := range 100 { + go func(i int) { + defer wg.Done() + _, err := client.RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "myactortype", + ActorId: strconv.Itoa(i), + Name: strconv.Itoa(i), + DueTime: "10000s", + }) + assert.NoError(t, err) + }(i) + go func(i int) { + defer wg.Done() + _, err := client.RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "myactortype2", + ActorId: strconv.Itoa(i), + Name: strconv.Itoa(i), + DueTime: "10000s", + }) + assert.NoError(t, err) + }(i) + } + wg.Wait() + + assert.Len(t, d.db.ActorReminders(t, ctx, "myactortype").Reminders, 100) + assert.Len(t, d.db.ActorReminders(t, ctx, "myactortype2").Reminders, 100) + assert.Empty(t, d.scheduler.EtcdJobs(t, ctx)) + + daprd1.Cleanup(t) + daprd2.Run(t, ctx) + daprd3.Run(t, ctx) + daprd4.Run(t, ctx) + daprd5.Run(t, ctx) + daprd2.WaitUntilRunning(t, ctx) + daprd3.WaitUntilRunning(t, ctx) + daprd4.WaitUntilRunning(t, ctx) + daprd5.WaitUntilRunning(t, ctx) + + assert.Len(t, d.db.ActorReminders(t, ctx, "myactortype").Reminders, 100) + assert.Len(t, d.db.ActorReminders(t, ctx, "myactortype2").Reminders, 100) + assert.Len(t, d.scheduler.EtcdJobs(t, ctx), 100) + + daprd6.Run(t, ctx) + daprd6.WaitUntilRunning(t, ctx) + + assert.Len(t, d.db.ActorReminders(t, ctx, "myactortype").Reminders, 100) + assert.Len(t, d.db.ActorReminders(t, ctx, "myactortype2").Reminders, 100) + assert.Len(t, d.scheduler.EtcdJobs(t, ctx), 200) + + daprd2.Cleanup(t) + daprd3.Cleanup(t) + daprd4.Cleanup(t) + daprd5.Cleanup(t) + daprd6.Cleanup(t) +} diff --git a/tests/integration/suite/actors/reminders/migration/scheduler/noop.go b/tests/integration/suite/actors/reminders/migration/scheduler/noop.go new file mode 100644 index 00000000000..bb812a06f20 --- /dev/null +++ b/tests/integration/suite/actors/reminders/migration/scheduler/noop.go @@ -0,0 +1,105 @@ +/* +Copyright 2024 The Dapr 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 scheduler + +import ( + "context" + "net/http" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/daprd" + "github.com/dapr/dapr/tests/integration/framework/process/http/app" + "github.com/dapr/dapr/tests/integration/framework/process/placement" + "github.com/dapr/dapr/tests/integration/framework/process/scheduler" + "github.com/dapr/dapr/tests/integration/framework/process/sqlite" + "github.com/dapr/dapr/tests/integration/suite" +) + +func init() { + suite.Register(new(noop)) +} + +type noop struct { + db *sqlite.SQLite + app *app.App + place *placement.Placement + scheduler *scheduler.Scheduler +} + +func (n *noop) Setup(t *testing.T) []framework.Option { + n.db = sqlite.New(t, sqlite.WithActorStateStore(true)) + n.app = app.New(t, + app.WithConfig(`{"entities": ["myactortype"]}`), + app.WithHandlerFunc("/actors/myactortype/myactorid", func(http.ResponseWriter, *http.Request) {}), + ) + n.scheduler = scheduler.New(t, scheduler.WithLogLevel("debug")) + n.place = placement.New(t) + + return []framework.Option{ + framework.WithProcesses(n.db, n.scheduler, n.place, n.app), + } +} + +func (n *noop) Run(t *testing.T, ctx context.Context) { + opts := []daprd.Option{ + daprd.WithResourceFiles(n.db.GetComponent(t)), + daprd.WithPlacementAddresses(n.place.Address()), + daprd.WithSchedulerAddresses(n.scheduler.Address()), + daprd.WithAppPort(n.app.Port()), + } + + daprd1 := daprd.New(t, append(opts, + daprd.WithConfigManifests(t, ` +apiVersion: dapr.io/v1alpha1 +kind: Configuration +metadata: + name: schedulerreminders +spec: + features: + - name: SchedulerReminders + enabled: false +`))...) + daprd2 := daprd.New(t, opts...) + + daprd1.Run(t, ctx) + daprd1.WaitUntilRunning(t, ctx) + client := daprd1.GRPCClient(t, ctx) + _, err := client.RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "myactortype", + ActorId: "myactorid", + Name: "myreminder", + DueTime: "10000s", + Period: "10000s", + Data: []byte("mydata"), + Ttl: "10000s", + }) + require.NoError(t, err) + resp, err := n.scheduler.ETCDClient(t).KV.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + require.NoError(t, err) + assert.Empty(t, resp.Kvs) + daprd1.Cleanup(t) + + daprd2.Run(t, ctx) + daprd2.WaitUntilRunning(t, ctx) + resp, err = n.scheduler.ETCDClient(t).KV.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + require.NoError(t, err) + assert.Empty(t, resp.Kvs) + daprd2.Cleanup(t) +} diff --git a/tests/integration/suite/actors/reminders/migration/scheduler/overwrite.go b/tests/integration/suite/actors/reminders/migration/scheduler/overwrite.go new file mode 100644 index 00000000000..158ab5a07d2 --- /dev/null +++ b/tests/integration/suite/actors/reminders/migration/scheduler/overwrite.go @@ -0,0 +1,216 @@ +/* +Copyright 2024 The Dapr 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 scheduler + +import ( + "context" + "encoding/base64" + "net/http" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + "google.golang.org/protobuf/types/known/anypb" + "google.golang.org/protobuf/types/known/wrapperspb" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + schedulerv1 "github.com/dapr/dapr/pkg/proto/scheduler/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/daprd" + "github.com/dapr/dapr/tests/integration/framework/process/http/app" + "github.com/dapr/dapr/tests/integration/framework/process/placement" + "github.com/dapr/dapr/tests/integration/framework/process/scheduler" + "github.com/dapr/dapr/tests/integration/framework/process/sqlite" + "github.com/dapr/dapr/tests/integration/suite" + "github.com/dapr/kit/ptr" +) + +func init() { + suite.Register(new(overwrite)) +} + +type overwrite struct { + db *sqlite.SQLite + app *app.App + place *placement.Placement + scheduler *scheduler.Scheduler +} + +func (o *overwrite) Setup(t *testing.T) []framework.Option { + o.db = sqlite.New(t, sqlite.WithActorStateStore(true)) + o.app = app.New(t, + app.WithConfig(`{"entities": ["myactortype"]}`), + app.WithHandlerFunc("/actors/myactortype/myactorid", func(http.ResponseWriter, *http.Request) {}), + ) + o.scheduler = scheduler.New(t, scheduler.WithLogLevel("debug")) + o.place = placement.New(t) + + return []framework.Option{ + framework.WithProcesses(o.db, o.scheduler, o.place, o.app), + } +} + +func (o *overwrite) Run(t *testing.T, ctx context.Context) { + opts := []daprd.Option{ + daprd.WithResourceFiles(o.db.GetComponent(t)), + daprd.WithPlacementAddresses(o.place.Address()), + daprd.WithSchedulerAddresses(o.scheduler.Address()), + daprd.WithAppPort(o.app.Port()), + } + + optsWithScheduler := []daprd.Option{ + daprd.WithResourceFiles(o.db.GetComponent(t)), + daprd.WithPlacementAddresses(o.place.Address()), + daprd.WithSchedulerAddresses(o.scheduler.Address()), + daprd.WithAppPort(o.app.Port()), + daprd.WithConfigManifests(t, ` +apiVersion: dapr.io/v1alpha1 +kind: Configuration +metadata: + name: schedulerreminders +spec: + features: + - name: SchedulerReminders + enabled: true +`), + } + + daprd1 := daprd.New(t, optsWithScheduler...) + daprd2 := daprd.New(t, opts...) + daprd3 := daprd.New(t, optsWithScheduler...) + + daprd1.Run(t, ctx) + daprd1.WaitUntilRunning(t, ctx) + client := daprd1.GRPCClient(t, ctx) + _, err := client.RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "myactortype", + ActorId: "myactorid", + Name: "myreminder", + DueTime: "10000s", + Period: "R100/PT10000S", + Data: []byte("mydata1"), + Ttl: "10000s", + }) + require.NoError(t, err) + sclient := o.scheduler.Client(t, ctx) + resp, err := sclient.ListJobs(ctx, &schedulerv1.ListJobsRequest{ + Metadata: &schedulerv1.JobMetadata{ + AppId: daprd1.AppID(), + Namespace: daprd1.Namespace(), + Target: &schedulerv1.JobTargetMetadata{ + Type: &schedulerv1.JobTargetMetadata_Actor{ + Actor: &schedulerv1.TargetActorReminder{ + Id: "myactorid", + Type: "myactortype", + }, + }, + }, + }, + }) + require.NoError(t, err) + + require.Len(t, resp.GetJobs(), 1) + njob := resp.GetJobs()[0] + assert.Equal(t, "myreminder", njob.GetName()) + expAny, err := anypb.New(wrapperspb.Bytes([]byte(`"` + base64.URLEncoding.EncodeToString([]byte("mydata1")) + `"`))) + require.NoError(t, err) + assert.Equal(t, &schedulerv1.Job{ + Schedule: ptr.Of("@every 2h46m40s"), + DueTime: ptr.Of("10000s"), + Ttl: ptr.Of("10000s"), + Data: expAny, + Repeats: ptr.Of(uint32(100)), + }, njob.GetJob()) + daprd1.Cleanup(t) + + daprd2.Run(t, ctx) + daprd2.WaitUntilRunning(t, ctx) + client = daprd2.GRPCClient(t, ctx) + _, err = client.RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "myactortype", + ActorId: "myactorid", + Name: "myreminder", + DueTime: "20000s", + Period: "R200/PT20000S", + Data: []byte("mydata2"), + Ttl: "20000s", + }) + require.NoError(t, err) + eresp, err := o.scheduler.ETCDClient(t).KV.Get(ctx, "dapr/jobs", clientv3.WithPrefix()) + require.NoError(t, err) + assert.Len(t, eresp.Kvs, 1) + + resp, err = sclient.ListJobs(ctx, &schedulerv1.ListJobsRequest{ + Metadata: &schedulerv1.JobMetadata{ + AppId: daprd2.AppID(), + Namespace: daprd2.Namespace(), + Target: &schedulerv1.JobTargetMetadata{ + Type: &schedulerv1.JobTargetMetadata_Actor{ + Actor: &schedulerv1.TargetActorReminder{ + Id: "myactorid", + Type: "myactortype", + }, + }, + }, + }, + }) + require.NoError(t, err) + require.Len(t, resp.GetJobs(), 1) + njob = resp.GetJobs()[0] + assert.Equal(t, "myreminder", njob.GetName()) + expAny, err = anypb.New(wrapperspb.Bytes([]byte(`"` + base64.URLEncoding.EncodeToString([]byte("mydata1")) + `"`))) + require.NoError(t, err) + assert.Equal(t, &schedulerv1.Job{ + Schedule: ptr.Of("@every 2h46m40s"), + DueTime: ptr.Of("10000s"), + Ttl: ptr.Of("10000s"), + Data: expAny, + Repeats: ptr.Of(uint32(100)), + }, njob.GetJob()) + daprd2.Cleanup(t) + + daprd3.Run(t, ctx) + daprd3.WaitUntilRunning(t, ctx) + resp, err = sclient.ListJobs(ctx, &schedulerv1.ListJobsRequest{ + Metadata: &schedulerv1.JobMetadata{ + AppId: daprd2.AppID(), + Namespace: daprd2.Namespace(), + Target: &schedulerv1.JobTargetMetadata{ + Type: &schedulerv1.JobTargetMetadata_Actor{ + Actor: &schedulerv1.TargetActorReminder{ + Id: "myactorid", + Type: "myactortype", + }, + }, + }, + }, + }) + require.NoError(t, err) + require.Len(t, resp.GetJobs(), 1) + njob = resp.GetJobs()[0] + assert.Equal(t, "myreminder", njob.GetName()) + expAny, err = anypb.New(wrapperspb.Bytes([]byte(`"` + base64.URLEncoding.EncodeToString([]byte("mydata2")) + `"`))) + require.NoError(t, err) + assert.Equal(t, "@every 5h33m20s", njob.GetJob().GetSchedule()) + assert.Equal(t, "20000s", njob.GetJob().GetDueTime()) + expTTL := time.Now().Add(20000 * time.Second) + gotTTL, err := time.Parse(time.RFC3339, njob.GetJob().GetTtl()) + require.NoError(t, err) + assert.InDelta(t, expTTL.UnixMilli(), gotTTL.UnixMilli(), float64(time.Second*10)) + assert.Equal(t, expAny, njob.GetJob().GetData()) + assert.Equal(t, uint32(200), njob.GetJob().GetRepeats()) + daprd3.Cleanup(t) +} diff --git a/tests/integration/suite/actors/reminders/migration/scheduler/rebalance.go b/tests/integration/suite/actors/reminders/migration/scheduler/rebalance.go new file mode 100644 index 00000000000..aa2fb7801fd --- /dev/null +++ b/tests/integration/suite/actors/reminders/migration/scheduler/rebalance.go @@ -0,0 +1,105 @@ +/* +Copyright 2024 The Dapr 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 scheduler + +import ( + "context" + "strconv" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/daprd/actors" + "github.com/dapr/dapr/tests/integration/framework/process/sqlite" + "github.com/dapr/dapr/tests/integration/suite" +) + +func init() { + suite.Register(new(rebalance)) +} + +type rebalance struct { + db *sqlite.SQLite + + actor1 *actors.Actors + actor2 *actors.Actors +} + +func (r *rebalance) Setup(t *testing.T) []framework.Option { + r.db = sqlite.New(t, + sqlite.WithActorStateStore(true), + sqlite.WithCreateStateTables(), + ) + r.actor1 = actors.New(t, + actors.WithDB(r.db), + actors.WithActorTypes("myactortype"), + ) + r.actor2 = actors.New(t, + actors.WithScheduler(r.actor1.Scheduler()), + actors.WithDB(r.db), + actors.WithActorTypes("myactortype"), + actors.WithPlacement(r.actor1.Placement()), + actors.WithFeatureSchedulerReminders(true), + ) + + return []framework.Option{ + framework.WithProcesses(r.db, r.actor1, r.actor2), + } +} + +func (r *rebalance) Run(t *testing.T, ctx context.Context) { + r.actor1.WaitUntilRunning(t, ctx) + r.actor2.WaitUntilRunning(t, ctx) + + assert.Empty(t, r.actor1.DB().ActorReminders(t, ctx, "myactortype").Reminders) + assert.Empty(t, r.actor2.DB().ActorReminders(t, ctx, "myactortype").Reminders) + assert.Empty(t, r.actor1.Scheduler().EtcdJobs(t, ctx)) + assert.Empty(t, r.actor2.Scheduler().EtcdJobs(t, ctx)) + + for i := range 200 { + _, err := r.actor1.GRPCClient(t, ctx).RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "myactortype", + ActorId: "initial-" + strconv.Itoa(i), + Name: "foo", + DueTime: "1000s", + }) + require.NoError(t, err) + } + + assert.Len(t, r.actor1.DB().ActorReminders(t, ctx, "myactortype").Reminders, 200) + assert.Len(t, r.actor2.DB().ActorReminders(t, ctx, "myactortype").Reminders, 200) + assert.Empty(t, r.actor1.Scheduler().EtcdJobs(t, ctx)) + assert.Empty(t, r.actor2.Scheduler().EtcdJobs(t, ctx)) + + t.Run("new daprd", func(t *testing.T) { + daprd := actors.New(t, + actors.WithDB(r.db), + actors.WithActorTypes("myactortype"), + actors.WithFeatureSchedulerReminders(false), + actors.WithPlacement(r.actor1.Placement()), + actors.WithScheduler(r.actor1.Scheduler()), + ) + daprd.Run(t, ctx) + daprd.WaitUntilRunning(t, ctx) + + assert.Len(t, r.actor1.DB().ActorReminders(t, ctx, "myactortype").Reminders, 200) + assert.Len(t, r.actor2.DB().ActorReminders(t, ctx, "myactortype").Reminders, 200) + assert.NotEmpty(t, r.actor1.Scheduler().EtcdJobs(t, ctx)) + assert.NotEmpty(t, r.actor2.Scheduler().EtcdJobs(t, ctx)) + daprd.Cleanup(t) + }) +} diff --git a/tests/integration/suite/actors/reminders/reminders.go b/tests/integration/suite/actors/reminders/reminders.go index b702191da9e..5af76ca76a9 100644 --- a/tests/integration/suite/actors/reminders/reminders.go +++ b/tests/integration/suite/actors/reminders/reminders.go @@ -14,6 +14,7 @@ limitations under the License. package reminders import ( + _ "github.com/dapr/dapr/tests/integration/suite/actors/reminders/migration" _ "github.com/dapr/dapr/tests/integration/suite/actors/reminders/scheduler" _ "github.com/dapr/dapr/tests/integration/suite/actors/reminders/serialization" ) diff --git a/tests/integration/suite/actors/reminders/scheduler/data.go b/tests/integration/suite/actors/reminders/scheduler/data.go new file mode 100644 index 00000000000..32015020554 --- /dev/null +++ b/tests/integration/suite/actors/reminders/scheduler/data.go @@ -0,0 +1,78 @@ +/* +Copyright 2024 The Dapr 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 scheduler + +import ( + "context" + "io" + "net/http" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/daprd/actors" + "github.com/dapr/dapr/tests/integration/suite" +) + +func init() { + suite.Register(new(data)) +} + +type data struct { + actors *actors.Actors + got chan string +} + +func (d *data) Setup(t *testing.T) []framework.Option { + d.got = make(chan string, 1) + d.actors = actors.New(t, + actors.WithFeatureSchedulerReminders(true), + actors.WithActorTypes("foo"), + actors.WithActorTypeHandler("foo", func(_ http.ResponseWriter, req *http.Request) { + got, err := io.ReadAll(req.Body) + assert.NoError(t, err) + d.got <- string(got) + }), + ) + + return []framework.Option{ + framework.WithProcesses(d.actors), + } +} + +func (d *data) Run(t *testing.T, ctx context.Context) { + d.actors.WaitUntilRunning(t, ctx) + + _, err := d.actors.GRPCClient(t, ctx).RegisterActorReminder(ctx, &rtv1.RegisterActorReminderRequest{ + ActorType: "foo", + ActorId: "1234", + Name: "helloworld", + DueTime: "0s", + Period: "1000s", + Ttl: "2000s", + Data: []byte("mydata"), + }) + require.NoError(t, err) + + select { + case got := <-d.got: + assert.JSONEq(t, `{"data":"bXlkYXRh","dueTime":"","period":""}`, got) + case <-time.After(10 * time.Second): + t.Fatal("timed out waiting for reminder") + } +} diff --git a/tests/integration/suite/actors/reminders/serialization/common.go b/tests/integration/suite/actors/reminders/serialization/common.go index e39660ea0a9..44612b19ce7 100644 --- a/tests/integration/suite/actors/reminders/serialization/common.go +++ b/tests/integration/suite/actors/reminders/serialization/common.go @@ -15,7 +15,7 @@ package serialization import ( "context" - "database/sql" + "fmt" "net/http" "strings" "sync/atomic" @@ -25,6 +25,8 @@ import ( chi "github.com/go-chi/chi/v5" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/dapr/dapr/tests/integration/framework/process/sqlite" ) func invokeActor(t *testing.T, ctx context.Context, baseURL string, client *http.Client) { @@ -51,10 +53,10 @@ func storeReminder(t *testing.T, ctx context.Context, baseURL string, client *ht require.Equal(t, http.StatusNoContent, resp.StatusCode) } -func loadRemindersFromDB(t *testing.T, ctx context.Context, db *sql.DB) (storedVal string) { +func loadRemindersFromDB(t *testing.T, ctx context.Context, db *sqlite.SQLite) (storedVal string) { queryCtx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() - err := db.QueryRowContext(queryCtx, "SELECT value FROM state WHERE key = 'actors||myactortype'").Scan(&storedVal) + err := db.GetConnection(t).QueryRowContext(queryCtx, fmt.Sprintf("SELECT value FROM %s WHERE key = 'actors||myactortype'", db.TableName())).Scan(&storedVal) require.NoError(t, err) return storedVal } diff --git a/tests/integration/suite/actors/reminders/serialization/default.go b/tests/integration/suite/actors/reminders/serialization/default.go index 6c313567c6a..fe214422bc0 100644 --- a/tests/integration/suite/actors/reminders/serialization/default.go +++ b/tests/integration/suite/actors/reminders/serialization/default.go @@ -83,7 +83,7 @@ func (d *defaultS) Run(t *testing.T, ctx context.Context) { // Check the data in the SQLite database // The value must begin with `[{`, which indicates it was serialized as JSON - storedVal := loadRemindersFromDB(t, ctx, d.db.GetConnection(t)) + storedVal := loadRemindersFromDB(t, ctx, d.db) assert.Truef(t, strings.HasPrefix(storedVal, "[{"), "Prefix not found in value: '%v'", storedVal) assert.Eventually(t, func() bool { diff --git a/tests/integration/suite/actors/reminders/serialization/json.go b/tests/integration/suite/actors/reminders/serialization/json.go index f7419997581..6edf170c57d 100644 --- a/tests/integration/suite/actors/reminders/serialization/json.go +++ b/tests/integration/suite/actors/reminders/serialization/json.go @@ -99,7 +99,7 @@ func (j *jsonFormat) Run(t *testing.T, ctx context.Context) { // Check the data in the SQLite database // The value must begin with `[{`, which indicates it was serialized as JSON - storedVal := loadRemindersFromDB(t, ctx, j.db.GetConnection(t)) + storedVal := loadRemindersFromDB(t, ctx, j.db) assert.Truef(t, strings.HasPrefix(storedVal, "[{"), "Prefix not found in value: '%v'", storedVal) // Ensure the reminder was invoked at least once diff --git a/tests/integration/suite/actors/reminders/serialization/protobuf.go b/tests/integration/suite/actors/reminders/serialization/protobuf.go index 1c65720a898..c0c2e4f40a7 100644 --- a/tests/integration/suite/actors/reminders/serialization/protobuf.go +++ b/tests/integration/suite/actors/reminders/serialization/protobuf.go @@ -63,6 +63,7 @@ func (p *protobufFormat) Setup(t *testing.T) []framework.Option { p.db = sqlite.New(t, sqlite.WithActorStateStore(true), sqlite.WithCreateStateTables(), + sqlite.WithTableName("state"), sqlite.WithExecs(fmt.Sprintf(` INSERT INTO state VALUES ('actors||myactortype','[{"registeredTime":"%[1]s","period":"2m","actorID":"myactorid","actorType":"myactortype","name":"oldreminder","dueTime":"0"}]',0,'e467f810-4e93-45ed-85d9-e68d9fc7af4a',NULL,'%[1]s'), @@ -107,7 +108,7 @@ func (p *protobufFormat) Run(t *testing.T, ctx context.Context) { // Check the data in the SQLite database // The value must be base64-encoded, and after being decoded it should begin with `\0pb`, which indicates it was serialized as protobuf - storedVal := loadRemindersFromDB(t, ctx, p.db.GetConnection(t)) + storedVal := loadRemindersFromDB(t, ctx, p.db) storedValBytes, err := base64.StdEncoding.DecodeString(storedVal) require.NoErrorf(t, err, "Failed to decode value from base64: '%v'", storedVal) assert.Truef(t, bytes.HasPrefix(storedValBytes, []byte{0, 'p', 'b'}), "Prefix not found in value: '%v'", storedVal) diff --git a/tests/integration/suite/daprd/metrics/http/pathmatching/high.go b/tests/integration/suite/daprd/metrics/http/pathmatching/high.go index d3572a4a788..0c27cfb3f16 100644 --- a/tests/integration/suite/daprd/metrics/http/pathmatching/high.go +++ b/tests/integration/suite/daprd/metrics/http/pathmatching/high.go @@ -88,13 +88,15 @@ func (h *highCardinality) Run(t *testing.T, ctx context.Context) { h.daprd.HTTPGet2xx(t, ctx, "/v1.0/invoke/myapp/method/orders/1234") h.daprd.HTTPGet2xx(t, ctx, "/v1.0/invoke/myapp/method/orders") h.daprd.HTTPGet2xx(t, ctx, "/v1.0/invoke/myapp/method/basket") - metrics := h.daprd.Metrics(t, ctx) - assert.Equal(t, 1, int(metrics["dapr_http_server_request_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/orders/{orderID}|status:200"])) - assert.Equal(t, 1, int(metrics["dapr_http_server_request_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/orders/1234|status:200"])) - assert.Equal(t, 1, int(metrics["dapr_http_server_request_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/orders|status:200"])) - assert.Equal(t, 1, int(metrics["dapr_http_server_request_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/basket|status:200"])) - assert.Equal(t, 1, int(metrics["dapr_http_server_response_count|app_id:myapp|method:GET|path:/v1.0/healthz|status:204"])) - assert.Equal(t, 1, int(metrics["dapr_http_server_response_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/orders/{orderID}|status:200"])) + assert.EventuallyWithT(t, func(c *assert.CollectT) { + metrics := h.daprd.Metrics(t, ctx) + assert.Equal(c, 1, int(metrics["dapr_http_server_request_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/orders/{orderID}|status:200"])) + assert.Equal(c, 1, int(metrics["dapr_http_server_request_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/orders/1234|status:200"])) + assert.Equal(c, 1, int(metrics["dapr_http_server_request_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/orders|status:200"])) + assert.Equal(c, 1, int(metrics["dapr_http_server_request_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/basket|status:200"])) + assert.Equal(c, 1, int(metrics["dapr_http_server_response_count|app_id:myapp|method:GET|path:/v1.0/healthz|status:204"])) + assert.Equal(c, 1, int(metrics["dapr_http_server_response_count|app_id:myapp|method:GET|path:/v1.0/invoke/myapp/method/orders/{orderID}|status:200"])) + }, time.Second*10, time.Millisecond*10) }) t.Run("service invocation - no match", func(t *testing.T) { From 12fafc0f0ca7b478c9f24a711e83368104b04cf7 Mon Sep 17 00:00:00 2001 From: joshvanl Date: Mon, 21 Oct 2024 12:14:00 +0100 Subject: [PATCH 17/27] Skip sqlite process on windows due to limitation Signed-off-by: joshvanl --- tests/integration/framework/process/sqlite/sqlite.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/framework/process/sqlite/sqlite.go b/tests/integration/framework/process/sqlite/sqlite.go index 49c293e528c..51b5387fec3 100644 --- a/tests/integration/framework/process/sqlite/sqlite.go +++ b/tests/integration/framework/process/sqlite/sqlite.go @@ -18,6 +18,7 @@ import ( "database/sql" "encoding/json" "path/filepath" + "runtime" "strconv" "sync" "testing" @@ -56,6 +57,10 @@ type SQLite struct { func New(t *testing.T, fopts ...Option) *SQLite { t.Helper() + if runtime.GOOS == "windows" { + t.Skip("Skipping test on Windows due to SQLite limitations") + } + opts := options{ name: "mystore", dbPath: filepath.Join(t.TempDir(), "test-data.db"), From 69e8e5339eaeeccdbff00b525b6429fbbad0590b Mon Sep 17 00:00:00 2001 From: joshvanl Date: Mon, 21 Oct 2024 16:27:31 +0100 Subject: [PATCH 18/27] Move htarget ready to outside of placement update ch Signed-off-by: joshvanl --- pkg/actors/actors.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/actors/actors.go b/pkg/actors/actors.go index d1391a168e2..257f1238049 100644 --- a/pkg/actors/actors.go +++ b/pkg/actors/actors.go @@ -323,7 +323,6 @@ func (a *actorsRuntime) Init(ctx context.Context) (err error) { a.placement.SetOnTableUpdateFn(func() { a.drainRebalancedActors() a.actorsReminders.OnPlacementTablesUpdated(ctx) - a.htarget.Ready() }) a.checker, err = a.getAppHealthChecker() @@ -360,6 +359,7 @@ func (a *actorsRuntime) Init(ctx context.Context) (err error) { }() log.Infof("Actor runtime started. Idle timeout: %v", a.actorsConfig.Config.ActorIdleTimeout) + a.htarget.Ready() return nil } From 1fa8e7c41cc3a7b4ddd22bf48def768076937c9d Mon Sep 17 00:00:00 2001 From: joshvanl Date: Tue, 22 Oct 2024 16:15:47 +0100 Subject: [PATCH 19/27] Fix migration integration test Signed-off-by: joshvanl --- pkg/actors/actors.go | 1 + pkg/actors/reminders/scheduler.go | 5 +++++ .../framework/process/http/app/app.go | 10 ++++++--- .../migration/scheduler/rebalance.go | 22 +++++++------------ 4 files changed, 21 insertions(+), 17 deletions(-) diff --git a/pkg/actors/actors.go b/pkg/actors/actors.go index 257f1238049..aaeb58f4479 100644 --- a/pkg/actors/actors.go +++ b/pkg/actors/actors.go @@ -251,6 +251,7 @@ func newActorsWithClock(opts ActorsOpts, clock clock.WithTicker) (ActorRuntime, AppID: opts.Config.AppID, ProviderOpts: providerOpts, ListActorTypesFn: a.Entities, + Healthz: opts.Healthz, }) } else { factory, err := opts.Config.GetRemindersProvider(a.placement) diff --git a/pkg/actors/reminders/scheduler.go b/pkg/actors/reminders/scheduler.go index 98d6abc7286..99f17057c91 100644 --- a/pkg/actors/reminders/scheduler.go +++ b/pkg/actors/reminders/scheduler.go @@ -26,6 +26,7 @@ import ( "github.com/dapr/dapr/pkg/actors/internal" "github.com/dapr/dapr/pkg/actors/reminders/migration" apierrors "github.com/dapr/dapr/pkg/api/errors" + "github.com/dapr/dapr/pkg/healthz" schedulerv1pb "github.com/dapr/dapr/pkg/proto/scheduler/v1" "github.com/dapr/dapr/pkg/runtime/scheduler/clients" "github.com/dapr/kit/ptr" @@ -38,6 +39,7 @@ type SchedulerOptions struct { Clients *clients.Clients ProviderOpts internal.ActorsProviderOptions ListActorTypesFn func() []string + Healthz healthz.Healthz } // Implements a reminders provider that does nothing when using Scheduler Service. @@ -48,6 +50,7 @@ type scheduler struct { lookUpActorFn internal.LookupActorFn stateReminder internal.RemindersProvider listActorTypesFn func() []string + htarget healthz.Target } func NewScheduler(opts SchedulerOptions) internal.RemindersProvider { @@ -57,6 +60,7 @@ func NewScheduler(opts SchedulerOptions) internal.RemindersProvider { appID: opts.AppID, stateReminder: NewStateStore(opts.ProviderOpts), listActorTypesFn: opts.ListActorTypesFn, + htarget: opts.Healthz.AddTarget(), } } @@ -86,6 +90,7 @@ func (s *scheduler) OnPlacementTablesUpdated(ctx context.Context) { if err != nil { log.Errorf("Error attempting to migrate reminders to scheduler: %s", err) } + s.htarget.Ready() } func (s *scheduler) DrainRebalancedReminders(actorType string, actorID string) { diff --git a/tests/integration/framework/process/http/app/app.go b/tests/integration/framework/process/http/app/app.go index 707efd99c5b..52bb1acd7c6 100644 --- a/tests/integration/framework/process/http/app/app.go +++ b/tests/integration/framework/process/http/app/app.go @@ -16,6 +16,7 @@ package app import ( "context" nethttp "net/http" + "sync" "sync/atomic" "testing" @@ -25,8 +26,9 @@ import ( type Option func(*options) type App struct { - http *http.HTTP - healthz *atomic.Bool + http *http.HTTP + healthz *atomic.Bool + cleanupOnce sync.Once } func New(t *testing.T, fopts ...Option) *App { @@ -72,7 +74,9 @@ func (a *App) Run(t *testing.T, ctx context.Context) { } func (a *App) Cleanup(t *testing.T) { - a.http.Cleanup(t) + a.cleanupOnce.Do(func() { + a.http.Cleanup(t) + }) } func (a *App) Port() int { diff --git a/tests/integration/suite/actors/reminders/migration/scheduler/rebalance.go b/tests/integration/suite/actors/reminders/migration/scheduler/rebalance.go index aa2fb7801fd..0dd150f00f3 100644 --- a/tests/integration/suite/actors/reminders/migration/scheduler/rebalance.go +++ b/tests/integration/suite/actors/reminders/migration/scheduler/rebalance.go @@ -24,7 +24,6 @@ import ( rtv1 "github.com/dapr/dapr/pkg/proto/runtime/v1" "github.com/dapr/dapr/tests/integration/framework" "github.com/dapr/dapr/tests/integration/framework/process/daprd/actors" - "github.com/dapr/dapr/tests/integration/framework/process/sqlite" "github.com/dapr/dapr/tests/integration/suite" ) @@ -33,31 +32,24 @@ func init() { } type rebalance struct { - db *sqlite.SQLite - actor1 *actors.Actors actor2 *actors.Actors } func (r *rebalance) Setup(t *testing.T) []framework.Option { - r.db = sqlite.New(t, - sqlite.WithActorStateStore(true), - sqlite.WithCreateStateTables(), - ) r.actor1 = actors.New(t, - actors.WithDB(r.db), actors.WithActorTypes("myactortype"), ) r.actor2 = actors.New(t, + actors.WithDB(r.actor1.DB()), actors.WithScheduler(r.actor1.Scheduler()), - actors.WithDB(r.db), actors.WithActorTypes("myactortype"), actors.WithPlacement(r.actor1.Placement()), actors.WithFeatureSchedulerReminders(true), ) return []framework.Option{ - framework.WithProcesses(r.db, r.actor1, r.actor2), + framework.WithProcesses(r.actor1, r.actor2), } } @@ -82,24 +74,26 @@ func (r *rebalance) Run(t *testing.T, ctx context.Context) { assert.Len(t, r.actor1.DB().ActorReminders(t, ctx, "myactortype").Reminders, 200) assert.Len(t, r.actor2.DB().ActorReminders(t, ctx, "myactortype").Reminders, 200) + assert.Empty(t, r.actor1.Scheduler().EtcdJobs(t, ctx)) - assert.Empty(t, r.actor2.Scheduler().EtcdJobs(t, ctx)) t.Run("new daprd", func(t *testing.T) { daprd := actors.New(t, - actors.WithDB(r.db), + actors.WithDB(r.actor1.DB()), actors.WithActorTypes("myactortype"), - actors.WithFeatureSchedulerReminders(false), + actors.WithFeatureSchedulerReminders(true), actors.WithPlacement(r.actor1.Placement()), actors.WithScheduler(r.actor1.Scheduler()), ) + t.Cleanup(func() { daprd.Cleanup(t) }) daprd.Run(t, ctx) daprd.WaitUntilRunning(t, ctx) assert.Len(t, r.actor1.DB().ActorReminders(t, ctx, "myactortype").Reminders, 200) assert.Len(t, r.actor2.DB().ActorReminders(t, ctx, "myactortype").Reminders, 200) + assert.Len(t, daprd.DB().ActorReminders(t, ctx, "myactortype").Reminders, 200) assert.NotEmpty(t, r.actor1.Scheduler().EtcdJobs(t, ctx)) - assert.NotEmpty(t, r.actor2.Scheduler().EtcdJobs(t, ctx)) + assert.NotEmpty(t, daprd.Scheduler().EtcdJobs(t, ctx)) daprd.Cleanup(t) }) } From 2b0614e83bd6966fad05843f3d7f33af28ae40e1 Mon Sep 17 00:00:00 2001 From: Manuel Menegazzo <65919883+m3nax@users.noreply.github.com> Date: Tue, 22 Oct 2024 18:54:21 +0200 Subject: [PATCH 20/27] Moved priority class declaration in the right position of manifests (#8205) Signed-off-by: Manuel Menegazzo <65919883+m3nax@users.noreply.github.com> Co-authored-by: Elena Kolevska Co-authored-by: Cassie Coyle --- .../templates/dapr_placement_statefulset.yaml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/charts/dapr/charts/dapr_placement/templates/dapr_placement_statefulset.yaml b/charts/dapr/charts/dapr_placement/templates/dapr_placement_statefulset.yaml index 8e08f99d88f..76aba737b93 100644 --- a/charts/dapr/charts/dapr_placement/templates/dapr_placement_statefulset.yaml +++ b/charts/dapr/charts/dapr_placement/templates/dapr_placement_statefulset.yaml @@ -253,6 +253,10 @@ spec: tolerations: {{ toYaml .Values.global.tolerations | indent 8 }} {{- end }} +{{- if .Values.global.priorityClassName }} + priorityClassName: +{{ toYaml .Values.global.priorityClassName | indent 8 }} +{{- end }} {{- if or (eq .Values.global.ha.enabled true) (eq .Values.ha true) }} {{- if eq .Values.cluster.forceInMemoryLog false }} volumeClaimTemplates: @@ -271,8 +275,4 @@ spec: {{- end }} {{- end }} {{- end }} -{{- if .Values.global.priorityClassName }} - priorityClassName: -{{ toYaml .Values.global.priorityClassName | indent 8 }} -{{- end }} {{- end }} From 81cd0a79ce7d1854e0d3599bc46b69c7e4045913 Mon Sep 17 00:00:00 2001 From: joshvanl Date: Tue, 22 Oct 2024 21:32:55 +0100 Subject: [PATCH 21/27] Adds actor reminder scheduler to actor state store Signed-off-by: joshvanl --- tests/config/dapr_postgres_state_actorstore.yaml | 1 + tests/integration/framework/process/logline/logline.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/config/dapr_postgres_state_actorstore.yaml b/tests/config/dapr_postgres_state_actorstore.yaml index 34057ecd7e0..cc85d4b418b 100644 --- a/tests/config/dapr_postgres_state_actorstore.yaml +++ b/tests/config/dapr_postgres_state_actorstore.yaml @@ -22,6 +22,7 @@ scopes: - actorfeatures - reentrantactor - actorreminder +- actorreminderscheduler - actorreminderpartition - actorinvocationapp - actormetadata-a diff --git a/tests/integration/framework/process/logline/logline.go b/tests/integration/framework/process/logline/logline.go index 5e4d3cd451c..8af668513c1 100644 --- a/tests/integration/framework/process/logline/logline.go +++ b/tests/integration/framework/process/logline/logline.go @@ -157,5 +157,5 @@ func (l *LogLine) Stderr() io.WriteCloser { } func (l *LogLine) EventuallyFoundAll(t *testing.T) { - assert.Eventually(t, l.FoundAll, time.Second*7, time.Millisecond*10) + assert.Eventually(t, l.FoundAll, time.Second*15, time.Millisecond*10) } From bbb81553e20aa88c2995e89e1a8aab4c8ec3b7b7 Mon Sep 17 00:00:00 2001 From: joshvanl Date: Tue, 22 Oct 2024 23:05:35 +0100 Subject: [PATCH 22/27] Always set actor runtime health ready on init Signed-off-by: joshvanl --- pkg/actors/actors.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/actors/actors.go b/pkg/actors/actors.go index aaeb58f4479..b34142923bb 100644 --- a/pkg/actors/actors.go +++ b/pkg/actors/actors.go @@ -301,6 +301,8 @@ func (a *actorsRuntime) Init(ctx context.Context) (err error) { return errors.New("actors runtime has already been closed") } + defer a.htarget.Ready() + if len(a.actorsConfig.ActorsService) == 0 { return errors.New("actors: couldn't connect to actors service: address is empty") } @@ -360,7 +362,6 @@ func (a *actorsRuntime) Init(ctx context.Context) (err error) { }() log.Infof("Actor runtime started. Idle timeout: %v", a.actorsConfig.Config.ActorIdleTimeout) - a.htarget.Ready() return nil } From 2b76820eb678ab1db5dabde50d9a77fc47bad01f Mon Sep 17 00:00:00 2001 From: David Wood Date: Wed, 23 Oct 2024 11:53:14 +0800 Subject: [PATCH 23/27] chore: fix some function names in comment (#8215) Signed-off-by: davidwoood Co-authored-by: Yaron Schneider --- pkg/config/configuration.go | 2 +- pkg/resiliency/noop.go | 2 +- pkg/runtime/trace.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/config/configuration.go b/pkg/config/configuration.go index 82c5ce3cea2..bf864165b41 100644 --- a/pkg/config/configuration.go +++ b/pkg/config/configuration.go @@ -678,7 +678,7 @@ func (c Configuration) GetLoggingSpec() LoggingSpec { return *c.Spec.LoggingSpec } -// GetLoggingSpec returns the Logging.APILogging spec. +// GetAPILoggingSpec returns the Logging.APILogging spec. // It's a short-hand that includes nil-checks for safety. func (c Configuration) GetAPILoggingSpec() APILoggingSpec { if c.Spec.LoggingSpec == nil || c.Spec.LoggingSpec.APILogging == nil { diff --git a/pkg/resiliency/noop.go b/pkg/resiliency/noop.go index aee0711b305..279da490c44 100644 --- a/pkg/resiliency/noop.go +++ b/pkg/resiliency/noop.go @@ -46,7 +46,7 @@ func (NoOp) ComponentOutboundPolicy(name string, componentName ComponentType) *P return nil } -// BuildInPolicy returns a NoOp policy definition for a built-in policy. +// BuiltInPolicy returns a NoOp policy definition for a built-in policy. func (NoOp) BuiltInPolicy(name BuiltInPolicyName) *PolicyDefinition { return nil } diff --git a/pkg/runtime/trace.go b/pkg/runtime/trace.go index 235e678fe53..82adf45b015 100644 --- a/pkg/runtime/trace.go +++ b/pkg/runtime/trace.go @@ -66,7 +66,7 @@ func (s *opentelemetryTracerProviderStore) RegisterSampler(sampler sdktrace.Samp s.sampler = sampler } -// RegisterTraceProvider registers a trace provider as per the tracer options in the store +// RegisterTracerProvider registers a trace provider as per the tracer options in the store func (s *opentelemetryTracerProviderStore) RegisterTracerProvider() *sdktrace.TracerProvider { if len(s.exporters) != 0 { tracerOptions := []sdktrace.TracerProviderOption{} From 488b4b6da479fcdcba0866f03739d2a0675d76af Mon Sep 17 00:00:00 2001 From: Yaron Schneider Date: Wed, 23 Oct 2024 10:24:44 -0700 Subject: [PATCH 24/27] Ensure dapr-api-token gets removed from metadata after validaton (#8219) * ensure dapr-api-token gets removed from metadata after validaton Signed-off-by: yaron2 * linter Signed-off-by: yaron2 --------- Signed-off-by: yaron2 Co-authored-by: Josh van Leeuwen --- pkg/api/grpc/auth.go | 32 +++-- .../framework/process/daprd/options.go | 6 + .../grpc/daprapitoken/remotenotoken.go | 111 ++++++++++++++++++ .../daprd/serviceinvocation/grpc/grpc.go | 1 + 4 files changed, 140 insertions(+), 10 deletions(-) create mode 100644 tests/integration/suite/daprd/serviceinvocation/grpc/daprapitoken/remotenotoken.go diff --git a/pkg/api/grpc/auth.go b/pkg/api/grpc/auth.go index 4f6e4e3635b..aab2208567f 100644 --- a/pkg/api/grpc/auth.go +++ b/pkg/api/grpc/auth.go @@ -5,43 +5,55 @@ import ( "net/http" "google.golang.org/grpc" + grpc_metadata "google.golang.org/grpc/metadata" "github.com/dapr/dapr/pkg/api/grpc/metadata" invokev1 "github.com/dapr/dapr/pkg/messaging/v1" ) +type wrappedStream struct { + grpc.ServerStream + ctx context.Context +} + +func (s *wrappedStream) Context() context.Context { + return s.ctx +} + func getAPIAuthenticationMiddlewares(apiToken, authHeader string) (grpc.UnaryServerInterceptor, grpc.StreamServerInterceptor) { return func(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (any, error) { - err := checkAPITokenInContext(ctx, apiToken, authHeader) + authCtx, err := checkAPITokenInContext(ctx, apiToken, authHeader) if err != nil { return nil, err } - return handler(ctx, req) + return handler(authCtx, req) }, func(srv any, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error { - err := checkAPITokenInContext(stream.Context(), apiToken, authHeader) + authCtx, err := checkAPITokenInContext(stream.Context(), apiToken, authHeader) if err != nil { return err } - return handler(srv, stream) + + return handler(srv, &wrappedStream{stream, authCtx}) } } // Checks if the API token in the gRPC request's context is valid; returns an error otherwise. -func checkAPITokenInContext(ctx context.Context, apiToken, authHeader string) error { +func checkAPITokenInContext(ctx context.Context, apiToken, authHeader string) (context.Context, error) { md, ok := metadata.FromIncomingContext(ctx) if !ok { - return invokev1.ErrorFromHTTPResponseCode(http.StatusUnauthorized, "missing metadata in request") + return ctx, invokev1.ErrorFromHTTPResponseCode(http.StatusUnauthorized, "missing metadata in request") } if len(md[authHeader]) == 0 { - return invokev1.ErrorFromHTTPResponseCode(http.StatusUnauthorized, "missing api token in request metadata") + return ctx, invokev1.ErrorFromHTTPResponseCode(http.StatusUnauthorized, "missing api token in request metadata") } if md[authHeader][0] != apiToken { - return invokev1.ErrorFromHTTPResponseCode(http.StatusUnauthorized, "authentication error: api token mismatch") + return ctx, invokev1.ErrorFromHTTPResponseCode(http.StatusUnauthorized, "authentication error: api token mismatch") } - md.Set(authHeader, "") - return nil + md.Delete(authHeader) + ctx = grpc_metadata.NewIncomingContext(ctx, md.Copy()) + return ctx, nil } diff --git a/tests/integration/framework/process/daprd/options.go b/tests/integration/framework/process/daprd/options.go index 519c070221c..b9f7ada8202 100644 --- a/tests/integration/framework/process/daprd/options.go +++ b/tests/integration/framework/process/daprd/options.go @@ -304,6 +304,12 @@ func WithAppAPIToken(t *testing.T, token string) Option { )) } +func WithDaprAPIToken(t *testing.T, token string) Option { + return WithExecOptions(exec.WithEnvVars(t, + "DAPR_API_TOKEN", token, + )) +} + func WithSentry(t *testing.T, sentry *sentry.Sentry) Option { return func(o *options) { WithExecOptions(exec.WithEnvVars(t, "DAPR_TRUST_ANCHORS", string(sentry.CABundle().TrustAnchors)))(o) diff --git a/tests/integration/suite/daprd/serviceinvocation/grpc/daprapitoken/remotenotoken.go b/tests/integration/suite/daprd/serviceinvocation/grpc/daprapitoken/remotenotoken.go new file mode 100644 index 00000000000..eaa5147e9b2 --- /dev/null +++ b/tests/integration/suite/daprd/serviceinvocation/grpc/daprapitoken/remotenotoken.go @@ -0,0 +1,111 @@ +/* +Copyright 2024 The Dapr 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 implieh. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package daprapitoken + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/metadata" + "google.golang.org/protobuf/types/known/anypb" + + commonv1 "github.com/dapr/dapr/pkg/proto/common/v1" + runtimev1 "github.com/dapr/dapr/pkg/proto/runtime/v1" + "github.com/dapr/dapr/tests/integration/framework" + "github.com/dapr/dapr/tests/integration/framework/process/daprd" + "github.com/dapr/dapr/tests/integration/framework/process/grpc/app" + testpb "github.com/dapr/dapr/tests/integration/framework/process/grpc/app/proto" + "github.com/dapr/dapr/tests/integration/suite" +) + +func init() { + suite.Register(new(remotenotoken)) +} + +type remotenotoken struct { + daprd1 *daprd.Daprd + daprd2 *daprd.Daprd + ch chan metadata.MD +} + +func (b *remotenotoken) Setup(t *testing.T) []framework.Option { + b.ch = make(chan metadata.MD, 1) + + app := app.New(t, + app.WithOnInvokeFn(func(ctx context.Context, _ *commonv1.InvokeRequest) (*commonv1.InvokeResponse, error) { + md, ok := metadata.FromIncomingContext(ctx) + require.True(t, ok) + b.ch <- md + return new(commonv1.InvokeResponse), nil + }), + app.WithPingFn(func(ctx context.Context, _ *testpb.PingRequest) (*testpb.PingResponse, error) { + md, _ := metadata.FromIncomingContext(ctx) + b.ch <- md + return new(testpb.PingResponse), nil + }), + ) + + b.daprd1 = daprd.New(t, + daprd.WithAppID("app1"), + daprd.WithAppProtocol("grpc"), + daprd.WithDaprAPIToken(t, "abc"), + ) + + b.daprd2 = daprd.New(t, + daprd.WithAppProtocol("grpc"), + daprd.WithAppPort(app.Port(t)), + ) + + return []framework.Option{ + framework.WithProcesses(app, b.daprd1, b.daprd2), + } +} + +func (b *remotenotoken) Run(t *testing.T, ctx context.Context) { + b.daprd1.WaitUntilRunning(t, ctx) + b.daprd2.WaitUntilRunning(t, ctx) + + client := testpb.NewTestServiceClient(b.daprd1.GRPCConn(t, ctx)) + ctx = metadata.AppendToOutgoingContext(ctx, "dapr-app-id", b.daprd2.AppID(), "dapr-api-token", "abc") + _, err := client.Ping(ctx, new(testpb.PingRequest)) + require.NoError(t, err) + + select { + case md := <-b.ch: + require.Empty(t, md.Get("dapr-api-token")) + case <-time.After(10 * time.Second): + assert.Fail(t, "timed out waiting for metadata") + } + + dclient := b.daprd1.GRPCClient(t, ctx) + _, err = dclient.InvokeService(ctx, &runtimev1.InvokeServiceRequest{ + Id: b.daprd2.AppID(), + Message: &commonv1.InvokeRequest{ + Method: "helloworld", + Data: new(anypb.Any), + HttpExtension: &commonv1.HTTPExtension{Verb: commonv1.HTTPExtension_GET}, + }, + }) + require.NoError(t, err) + + select { + case md := <-b.ch: + require.Empty(t, md.Get("dapr-api-token")) + case <-time.After(5 * time.Second): + assert.Fail(t, "timed out waiting for metadata") + } +} diff --git a/tests/integration/suite/daprd/serviceinvocation/grpc/grpc.go b/tests/integration/suite/daprd/serviceinvocation/grpc/grpc.go index 32916457ead..383597fb6df 100644 --- a/tests/integration/suite/daprd/serviceinvocation/grpc/grpc.go +++ b/tests/integration/suite/daprd/serviceinvocation/grpc/grpc.go @@ -15,4 +15,5 @@ package grpc import ( _ "github.com/dapr/dapr/tests/integration/suite/daprd/serviceinvocation/grpc/appapitoken" + _ "github.com/dapr/dapr/tests/integration/suite/daprd/serviceinvocation/grpc/daprapitoken" ) From 297bff5aef5dc929f0f5a94d5836eecb619ffac2 Mon Sep 17 00:00:00 2001 From: joshvanl Date: Thu, 24 Oct 2024 14:29:23 +0100 Subject: [PATCH 25/27] temp disable scheduler actor reminder e2e test before staging queue added Signed-off-by: joshvanl --- .../e2e/actor_reminder/actor_reminder_test.go | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/e2e/actor_reminder/actor_reminder_test.go b/tests/e2e/actor_reminder/actor_reminder_test.go index 23ec350f8cd..9690cdcf241 100644 --- a/tests/e2e/actor_reminder/actor_reminder_test.go +++ b/tests/e2e/actor_reminder/actor_reminder_test.go @@ -140,22 +140,22 @@ func TestMain(m *testing.M) { "TEST_APP_ACTOR_TYPE": actorName, }, }, - { - AppName: appName + "scheduler", - DaprEnabled: true, - DebugLoggingEnabled: true, - ImageName: "e2e-actorfeatures", - Config: "featureactorreminderscheduler", - Replicas: 1, - IngressEnabled: true, - DaprCPULimit: "2.0", - DaprCPURequest: "0.1", - AppCPULimit: "2.0", - AppCPURequest: "0.1", - AppEnv: map[string]string{ - "TEST_APP_ACTOR_TYPE": actorName, - }, - }, + //{ + // AppName: appName + "scheduler", + // DaprEnabled: true, + // DebugLoggingEnabled: true, + // ImageName: "e2e-actorfeatures", + // Config: "featureactorreminderscheduler", + // Replicas: 1, + // IngressEnabled: true, + // DaprCPULimit: "2.0", + // DaprCPURequest: "0.1", + // AppCPULimit: "2.0", + // AppCPURequest: "0.1", + // AppEnv: map[string]string{ + // "TEST_APP_ACTOR_TYPE": actorName, + // }, + //}, } tr = runner.NewTestRunner(appName, testApps, nil, nil) From acc274f94d74a6919b99d9c55a7137283e6bb77c Mon Sep 17 00:00:00 2001 From: joshvanl Date: Thu, 24 Oct 2024 14:40:21 +0100 Subject: [PATCH 26/27] Adds cleanup to distributed run Signed-off-by: joshvanl --- .../actors/reminders/migration/scheduler/distributed.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/integration/suite/actors/reminders/migration/scheduler/distributed.go b/tests/integration/suite/actors/reminders/migration/scheduler/distributed.go index 7a3d1be6c2e..881601c1d52 100644 --- a/tests/integration/suite/actors/reminders/migration/scheduler/distributed.go +++ b/tests/integration/suite/actors/reminders/migration/scheduler/distributed.go @@ -102,6 +102,7 @@ spec: daprd6 := daprd.New(t, optsApp1WithScheduler...) daprd1.Run(t, ctx) + t.Cleanup(func() { daprd1.Cleanup(t) }) daprd1.WaitUntilRunning(t, ctx) client := daprd1.GRPCClient(t, ctx) @@ -136,10 +137,16 @@ spec: assert.Empty(t, d.scheduler.EtcdJobs(t, ctx)) daprd1.Cleanup(t) + daprd2.Run(t, ctx) + t.Cleanup(func() { daprd2.Cleanup(t) }) daprd3.Run(t, ctx) + t.Cleanup(func() { daprd3.Cleanup(t) }) daprd4.Run(t, ctx) + t.Cleanup(func() { daprd4.Cleanup(t) }) daprd5.Run(t, ctx) + t.Cleanup(func() { daprd5.Cleanup(t) }) + daprd2.WaitUntilRunning(t, ctx) daprd3.WaitUntilRunning(t, ctx) daprd4.WaitUntilRunning(t, ctx) @@ -150,6 +157,7 @@ spec: assert.Len(t, d.scheduler.EtcdJobs(t, ctx), 100) daprd6.Run(t, ctx) + t.Cleanup(func() { daprd6.Cleanup(t) }) daprd6.WaitUntilRunning(t, ctx) assert.Len(t, d.db.ActorReminders(t, ctx, "myactortype").Reminders, 100) From 92efe9496d0c72d7c2bccf0ca0de5931e9111a9f Mon Sep 17 00:00:00 2001 From: Patrick Assuied Date: Thu, 24 Oct 2024 15:36:00 -0700 Subject: [PATCH 27/27] Provide ability to configure dapr grpc ports through dapr pod annotations (#8228) * Provide ability to configure dapr grpc ports through dapr pod annotations Signed-off-by: Patrick Assuied * addressing static check issues Signed-off-by: Patrick Assuied * Added tests for GRPC port overrides Signed-off-by: Patrick Assuied * cleanup and more tests Signed-off-by: Patrick Assuied * cleanup Signed-off-by: Patrick Assuied * addressed linting issues Signed-off-by: Patrick Assuied * Addressed PR feedback Signed-off-by: Patrick Assuied * linting :( Signed-off-by: Patrick Assuied * removing unnecessary ignore Signed-off-by: Patrick Assuied --------- Signed-off-by: Patrick Assuied Co-authored-by: Yaron Schneider --- pkg/injector/annotations/annotations.go | 2 + pkg/injector/patcher/sidecar.go | 4 +- .../patcher/sidecar_container_test.go | 64 ++++++++++ pkg/operator/handlers/dapr_handler.go | 69 +++++------ pkg/operator/handlers/dapr_handler_test.go | 112 +++++++++++++++++- pkg/operator/meta/meta.go | 12 ++ 6 files changed, 224 insertions(+), 39 deletions(-) diff --git a/pkg/injector/annotations/annotations.go b/pkg/injector/annotations/annotations.go index a1e61bdc896..c6249c1c6cc 100644 --- a/pkg/injector/annotations/annotations.go +++ b/pkg/injector/annotations/annotations.go @@ -35,6 +35,8 @@ const ( KeyEnableDebug = "dapr.io/enable-debug" KeyDebugPort = "dapr.io/debug-port" KeyEnv = "dapr.io/env" + KeyAPIGRPCPort = "dapr.io/grpc-port" + KeyInternalGRPCPort = "dapr.io/internal-grpc-port" KeyCPURequest = "dapr.io/sidecar-cpu-request" KeyCPULimit = "dapr.io/sidecar-cpu-limit" KeyMemoryRequest = "dapr.io/sidecar-memory-request" diff --git a/pkg/injector/patcher/sidecar.go b/pkg/injector/patcher/sidecar.go index 77528a60dac..d2e86780c41 100644 --- a/pkg/injector/patcher/sidecar.go +++ b/pkg/injector/patcher/sidecar.go @@ -56,8 +56,6 @@ type SidecarConfig struct { RemindersService string SentrySPIFFEID string SidecarHTTPPort int32 `default:"3500"` - SidecarAPIGRPCPort int32 `default:"50001"` - SidecarInternalGRPCPort int32 `default:"50002"` SidecarPublicPort int32 `default:"3501"` Enabled bool `annotation:"dapr.io/enabled"` @@ -77,6 +75,8 @@ type SidecarConfig struct { EnableDebug bool `annotation:"dapr.io/enable-debug" default:"false"` SidecarDebugPort int32 `annotation:"dapr.io/debug-port" default:"40000"` Env string `annotation:"dapr.io/env"` + SidecarAPIGRPCPort int32 `annotation:"dapr.io/grpc-port" default:"50001"` + SidecarInternalGRPCPort int32 `annotation:"dapr.io/internal-grpc-port" default:"50002"` SidecarCPURequest string `annotation:"dapr.io/sidecar-cpu-request"` SidecarCPULimit string `annotation:"dapr.io/sidecar-cpu-limit"` SidecarMemoryRequest string `annotation:"dapr.io/sidecar-memory-request"` diff --git a/pkg/injector/patcher/sidecar_container_test.go b/pkg/injector/patcher/sidecar_container_test.go index 019bbfda3f3..251aa8d656a 100644 --- a/pkg/injector/patcher/sidecar_container_test.go +++ b/pkg/injector/patcher/sidecar_container_test.go @@ -319,6 +319,7 @@ func TestGetSidecarContainer(t *testing.T) { }, }, }) + //nolint:goconst c.SidecarImage = "daprio/dapr" c.ImagePullPolicy = "Always" c.Namespace = "dapr-system" @@ -367,6 +368,69 @@ func TestGetSidecarContainer(t *testing.T) { assert.Equal(t, corev1.PullAlways, container.ImagePullPolicy) }) + t.Run("get sidecar container with custom grpc ports", func(t *testing.T) { + c := NewSidecarConfig(&corev1.Pod{ + ObjectMeta: metav1.ObjectMeta{ + Annotations: map[string]string{ + annotations.KeyAppID: "app_id", + annotations.KeyConfig: "config", + annotations.KeyAppPort: "5000", + annotations.KeyLogAsJSON: "true", + annotations.KeyAPITokenSecret: "secret", + annotations.KeyAppTokenSecret: "appsecret", + annotations.KeyAPIGRPCPort: "12345", + annotations.KeyInternalGRPCPort: "12346", + }, + }, + }) + c.SidecarImage = "daprio/dapr" + c.ImagePullPolicy = "Always" + c.Namespace = "dapr-system" + c.OperatorAddress = "controlplane:9000" + c.PlacementAddress = "placement:50000" + c.SentryAddress = "sentry:50000" + c.MTLSEnabled = true + c.Identity = "pod_identity" + c.ControlPlaneNamespace = "my-namespace" + c.ControlPlaneTrustDomain = "test.example.com" + + c.SetFromPodAnnotations() + + container, err := c.getSidecarContainer(getSidecarContainerOpts{}) + require.NoError(t, err) + + expectedArgs := []string{ + "/daprd", + "--dapr-http-port", "3500", + "--dapr-grpc-port", "12345", + "--dapr-internal-grpc-port", "12346", + "--dapr-listen-addresses", "[::1],127.0.0.1", + "--dapr-public-port", "3501", + "--app-id", "app_id", + "--app-protocol", "http", + "--log-level", "info", + "--dapr-graceful-shutdown-seconds", "-1", + "--mode", "kubernetes", + "--control-plane-address", "controlplane:9000", + "--sentry-address", "sentry:50000", + "--app-port", "5000", + "--enable-metrics", + "--metrics-port", "9090", + "--config", "config", + "--placement-host-address", "placement:50000", + "--log-as-json", + "--enable-mtls", + } + + // Command should be empty, image's entrypoint to be used. + assert.Empty(t, container.Command) + assertEqualJSON(t, container.Env, `[{"name":"NAMESPACE","value":"dapr-system"},{"name":"DAPR_TRUST_ANCHORS"},{"name":"POD_NAME","valueFrom":{"fieldRef":{"fieldPath":"metadata.name"}}},{"name":"DAPR_CONTROLPLANE_NAMESPACE","value":"my-namespace"},{"name":"DAPR_CONTROLPLANE_TRUST_DOMAIN","value":"test.example.com"},{"name":"DAPR_API_TOKEN","valueFrom":{"secretKeyRef":{"name":"secret","key":"token"}}},{"name":"APP_API_TOKEN","valueFrom":{"secretKeyRef":{"name":"appsecret","key":"token"}}}]`) + // default image + assert.Equal(t, "daprio/dapr", container.Image) + assert.EqualValues(t, expectedArgs, container.Args) + assert.Equal(t, corev1.PullAlways, container.ImagePullPolicy) + }) + t.Run("get sidecar container with debugging", func(t *testing.T) { c := NewSidecarConfig(&corev1.Pod{ ObjectMeta: metav1.ObjectMeta{ diff --git a/pkg/operator/handlers/dapr_handler.go b/pkg/operator/handlers/dapr_handler.go index 8fbbe122ce3..0999b160617 100644 --- a/pkg/operator/handlers/dapr_handler.go +++ b/pkg/operator/handlers/dapr_handler.go @@ -26,21 +26,21 @@ import ( ) const ( - daprSidecarHTTPPortName = "dapr-http" - daprSidecarAPIGRPCPortName = "dapr-grpc" - daprSidecarInternalGRPCPortName = "dapr-internal" - daprSidecarMetricsPortName = "dapr-metrics" - daprSidecarHTTPPort = 3500 - daprSidecarAPIGRPCPort = 50001 - daprSidecarInternalGRPCPort = 50002 - defaultMetricsEnabled = true - defaultMetricsPort = 9090 - clusterIPNone = "None" - daprServiceOwnerField = ".metadata.controller" - annotationPrometheusProbe = "prometheus.io/probe" - annotationPrometheusScrape = "prometheus.io/scrape" - annotationPrometheusPort = "prometheus.io/port" - annotationPrometheusPath = "prometheus.io/path" + daprSidecarHTTPPortName = "dapr-http" + daprSidecarAPIGRPCPortName = "dapr-grpc" + daprSidecarInternalGRPCPortName = "dapr-internal" + daprSidecarMetricsPortName = "dapr-metrics" + daprSidecarHTTPPort = 3500 + daprSidecarDefaultAPIGRPCPort = 50001 + daprSidecarDefaultInternalGRPCPort = 50002 + defaultMetricsEnabled = true + defaultMetricsPort = 9090 + clusterIPNone = "None" + daprServiceOwnerField = ".metadata.controller" + annotationPrometheusProbe = "prometheus.io/probe" + annotationPrometheusScrape = "prometheus.io/scrape" + annotationPrometheusPort = "prometheus.io/port" + annotationPrometheusPath = "prometheus.io/path" ) var log = logger.NewLogger("dapr.operator.handlers") @@ -274,10 +274,12 @@ func (h *DaprHandler) createDaprServiceValues(ctx context.Context, expectedServi if enableMetrics { annotationsMap[annotationPrometheusProbe] = "true" annotationsMap[annotationPrometheusScrape] = "true" // WARN: deprecated as of v1.7 please use prometheus.io/probe instead. - annotationsMap[annotationPrometheusPort] = strconv.Itoa(metricsPort) + annotationsMap[annotationPrometheusPort] = strconv.FormatInt(int64(metricsPort), 10) annotationsMap[annotationPrometheusPath] = "/" } + grpcPort := h.getGRPCPort(wrapper) + internalGRPCPort := h.getInternalGRPCPort(wrapper) return &corev1.Service{ ObjectMeta: metaV1.ObjectMeta{ Name: expectedService.Name, @@ -297,22 +299,20 @@ func (h *DaprHandler) createDaprServiceValues(ctx context.Context, expectedServi }, { Protocol: corev1.ProtocolTCP, - Port: int32(daprSidecarAPIGRPCPort), - TargetPort: intstr.FromInt(daprSidecarAPIGRPCPort), + Port: grpcPort, + TargetPort: intstr.FromInt32(grpcPort), Name: daprSidecarAPIGRPCPortName, }, { Protocol: corev1.ProtocolTCP, - Port: int32(daprSidecarInternalGRPCPort), - TargetPort: intstr.FromInt(daprSidecarInternalGRPCPort), + Port: internalGRPCPort, + TargetPort: intstr.FromInt32(internalGRPCPort), Name: daprSidecarInternalGRPCPortName, }, { - Protocol: corev1.ProtocolTCP, - // TODO: update types - //nolint:gosec - Port: int32(metricsPort), - TargetPort: intstr.FromInt(metricsPort), + Protocol: corev1.ProtocolTCP, + Port: metricsPort, + TargetPort: intstr.FromInt32(metricsPort), Name: daprSidecarMetricsPortName, }, }, @@ -338,15 +338,16 @@ func (h *DaprHandler) getEnableMetrics(wrapper ObjectWrapper) bool { return enableMetrics } -func (h *DaprHandler) getMetricsPort(wrapper ObjectWrapper) int { - annotationsMap := wrapper.GetTemplateAnnotations() - metricsPort := defaultMetricsPort - if val := annotationsMap[annotations.KeyMetricsPort]; val != "" { - if v, err := strconv.Atoi(val); err == nil { - metricsPort = v - } - } - return metricsPort +func (h *DaprHandler) getMetricsPort(wrapper ObjectWrapper) int32 { + return meta.GetAnnotationIntValueOrDefault(wrapper.GetTemplateAnnotations(), annotations.KeyMetricsPort, defaultMetricsPort) +} + +func (h *DaprHandler) getGRPCPort(wrapper ObjectWrapper) int32 { + return meta.GetAnnotationIntValueOrDefault(wrapper.GetTemplateAnnotations(), annotations.KeyAPIGRPCPort, daprSidecarDefaultAPIGRPCPort) +} + +func (h *DaprHandler) getInternalGRPCPort(wrapper ObjectWrapper) int32 { + return meta.GetAnnotationIntValueOrDefault(wrapper.GetTemplateAnnotations(), annotations.KeyInternalGRPCPort, daprSidecarDefaultInternalGRPCPort) } func (h *DaprHandler) isReconciled(owner *metaV1.OwnerReference) bool { diff --git a/pkg/operator/handlers/dapr_handler_test.go b/pkg/operator/handlers/dapr_handler_test.go index 618931fc73c..e380e317fe0 100644 --- a/pkg/operator/handlers/dapr_handler_test.go +++ b/pkg/operator/handlers/dapr_handler_test.go @@ -13,6 +13,7 @@ import ( metaV1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/runtime" "k8s.io/apimachinery/pkg/types" + "k8s.io/apimachinery/pkg/util/intstr" "k8s.io/client-go/kubernetes/scheme" "sigs.k8s.io/controller-runtime/pkg/client/fake" @@ -133,6 +134,26 @@ func TestCreateDaprServiceAppIDAndMetricsSettings(t *testing.T) { assert.Equal(t, "", service.ObjectMeta.Annotations["prometheus.io/path"]) } +func TestCreateDaprServiceAppIDAndPortsOverride(t *testing.T) { + testDaprHandler := getTestDaprHandler() + ctx := context.Background() + myDaprService := types.NamespacedName{ + Namespace: "test", + Name: "test", + } + deployment := getDeployment("test", "true") + deployment.GetTemplateAnnotations()[annotations.KeyAPIGRPCPort] = "12345" + deployment.GetTemplateAnnotations()[annotations.KeyInternalGRPCPort] = "12346" + + service := testDaprHandler.createDaprServiceValues(ctx, myDaprService, deployment, "test") + require.NotNil(t, service) + assert.Equal(t, "test", service.ObjectMeta.Annotations[annotations.KeyAppID]) + assert.Equal(t, int32(12345), service.Spec.Ports[1].Port) + assert.Equal(t, intstr.FromInt(12345), service.Spec.Ports[1].TargetPort) + assert.Equal(t, int32(12346), service.Spec.Ports[2].Port) + assert.Equal(t, intstr.FromInt(12346), service.Spec.Ports[2].TargetPort) +} + func TestPatchDaprService(t *testing.T) { testDaprHandler := getTestDaprHandler() @@ -175,6 +196,78 @@ func TestPatchDaprService(t *testing.T) { assert.Equal(t, "app", actualService.OwnerReferences[0].Name) } +func TestGetGRPCPort(t *testing.T) { + testDaprHandler := getTestDaprHandler() + t.Run("GRPC port override", func(t *testing.T) { + // Arrange + grpcPort := "12345" + deployment := getDeploymentWithGRPCPortAnnotation("test_id", "true", &grpcPort, nil) + + // Act + p := testDaprHandler.getGRPCPort(deployment) + + // Assert + assert.Equal(t, int32(12345), p) + }) + t.Run("invalid GRPC port override", func(t *testing.T) { + // Arrange + grpcPort := "XXXX" + deployment := getDeploymentWithGRPCPortAnnotation("test_id", "true", &grpcPort, nil) + + // Act + p := testDaprHandler.getGRPCPort(deployment) + + // Assert + assert.Equal(t, int32(daprSidecarDefaultAPIGRPCPort), p) + }) + t.Run("no GRPC port override", func(t *testing.T) { + // Arrange + deployment := getDeployment("test_id", "true") + + // Act + p := testDaprHandler.getGRPCPort(deployment) + + // Assert + assert.Equal(t, int32(daprSidecarDefaultAPIGRPCPort), p) + }) +} + +func TestGetInternalGRPCPort(t *testing.T) { + testDaprHandler := getTestDaprHandler() + t.Run("Internal GRPC port override", func(t *testing.T) { + // Arrange + grpcPort := "12345" + deployment := getDeploymentWithGRPCPortAnnotation("test_id", "true", nil, &grpcPort) + + // Act + p := testDaprHandler.getInternalGRPCPort(deployment) + + // Assert + assert.Equal(t, int32(12345), p) + }) + t.Run("invalid Internal GRPC port override", func(t *testing.T) { + // Arrange + grpcPort := "XXXX" + deployment := getDeploymentWithGRPCPortAnnotation("test_id", "true", nil, &grpcPort) + + // Act + p := testDaprHandler.getInternalGRPCPort(deployment) + + // Assert + assert.Equal(t, int32(daprSidecarDefaultInternalGRPCPort), p) + }) + t.Run("no Internal GRPC port override", func(t *testing.T) { + // Arrange + deployment := getDeployment("test_id", "true") + + // Act + p := testDaprHandler.getInternalGRPCPort(deployment) + + // Assert + assert.Equal(t, int32(daprSidecarDefaultInternalGRPCPort), p) + }) +} + func TestGetMetricsPort(t *testing.T) { testDaprHandler := getTestDaprHandler() t.Run("metrics port override", func(t *testing.T) { @@ -185,7 +278,7 @@ func TestGetMetricsPort(t *testing.T) { p := testDaprHandler.getMetricsPort(deployment) // Assert - assert.Equal(t, 5050, p) + assert.Equal(t, int32(5050), p) }) t.Run("invalid metrics port override", func(t *testing.T) { // Arrange @@ -195,7 +288,7 @@ func TestGetMetricsPort(t *testing.T) { p := testDaprHandler.getMetricsPort(deployment) // Assert - assert.Equal(t, defaultMetricsPort, p) + assert.Equal(t, int32(defaultMetricsPort), p) }) t.Run("no metrics port override", func(t *testing.T) { // Arrange @@ -205,7 +298,7 @@ func TestGetMetricsPort(t *testing.T) { p := testDaprHandler.getMetricsPort(deployment) // Assert - assert.Equal(t, defaultMetricsPort, p) + assert.Equal(t, int32(defaultMetricsPort), p) }) } @@ -315,6 +408,19 @@ func getDeploymentWithMetricsPortAnnotation(daprID string, daprEnabled string, m return d } +func getDeploymentWithGRPCPortAnnotation(daprID string, daprEnabled string, grpcPort *string, internalGRPCPort *string) ObjectWrapper { + d := getDeployment(daprID, daprEnabled) + + if grpcPort != nil { + d.GetTemplateAnnotations()[annotations.KeyAPIGRPCPort] = *grpcPort + } + + if internalGRPCPort != nil { + d.GetTemplateAnnotations()[annotations.KeyInternalGRPCPort] = *internalGRPCPort + } + return d +} + func getDeployment(appID string, daprEnabled string) ObjectWrapper { return &DeploymentWrapper{testobjects.GetDeployment(appID, daprEnabled)} } diff --git a/pkg/operator/meta/meta.go b/pkg/operator/meta/meta.go index 6af1563a330..e682da55f36 100644 --- a/pkg/operator/meta/meta.go +++ b/pkg/operator/meta/meta.go @@ -1,6 +1,8 @@ package meta import ( + "strconv" + "github.com/dapr/dapr/pkg/injector/annotations" injectorConsts "github.com/dapr/dapr/pkg/injector/consts" "github.com/dapr/kit/utils" @@ -11,6 +13,16 @@ func IsAnnotatedForDapr(a map[string]string) bool { return utils.IsTruthy(a[annotations.KeyEnabled]) } +func GetAnnotationIntValueOrDefault(a map[string]string, annotationKey string, defaultValue int32) int32 { + // return value of annotation if exists, otherwise return default value + if value := a[annotationKey]; value != "" { + if val, err := strconv.ParseInt(value, 10, 32); err == nil { + return int32(val) + } + } + return defaultValue +} + // IsSidecarPresent whether the daprd sidecar is present, either because injector added it or because the user did. func IsSidecarPresent(labels map[string]string) bool { if _, ok := labels[injectorConsts.SidecarInjectedLabel]; ok {